-
Notifications
You must be signed in to change notification settings - Fork 28.5k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SPARK-6328] [Python] Python API for StreamingListener #9186
Changes from 12 commits
a63ef33
5b3da81
f6c91e9
1e6a87e
2579f64
afad086
3523e1f
7d50848
9d66e81
aa87c40
0ac3df6
233104d
79d70ab
47c12ed
f1c28c6
6601913
9e4e04a
5415389
7cdaf37
ddad255
64192d6
5349c82
c941c3e
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,192 @@ | ||
# | ||
# Licensed to the Apache Software Foundation (ASF) under one or more | ||
# contributor license agreements. See the NOTICE file distributed with | ||
# this work for additional information regarding copyright ownership. | ||
# The ASF licenses this file to You under the Apache License, Version 2.0 | ||
# (the "License"); you may not use this file except in compliance with | ||
# the License. You may obtain a copy of the License at | ||
# | ||
# http://www.apache.org/licenses/LICENSE-2.0 | ||
# | ||
# Unless required by applicable law or agreed to in writing, software | ||
# distributed under the License is distributed on an "AS IS" BASIS, | ||
# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
# See the License for the specific language governing permissions and | ||
# limitations under the License. | ||
# | ||
|
||
__all__ = ["StreamingListener"] | ||
|
||
|
||
class StreamingListener(object): | ||
|
||
def __init__(self): | ||
pass | ||
|
||
def onReceiverStarted(self, receiverStarted): | ||
""" | ||
Called when a receiver has been started | ||
""" | ||
pass | ||
|
||
def onReceiverError(self, receiverError): | ||
""" | ||
Called when a receiver has reported an error | ||
""" | ||
pass | ||
|
||
def onReceiverStopped(self, receiverStopped): | ||
""" | ||
Called when a receiver has been stopped | ||
""" | ||
pass | ||
|
||
def onBatchSubmitted(self, batchSubmitted): | ||
""" | ||
Called when a batch of jobs has been submitted for processing. | ||
""" | ||
pass | ||
|
||
def onBatchStarted(self, batchStarted): | ||
""" | ||
Called when processing of a batch of jobs has started. | ||
""" | ||
pass | ||
|
||
def onBatchCompleted(self, batchCompleted): | ||
""" | ||
Called when processing of a batch of jobs has completed. | ||
""" | ||
pass | ||
|
||
def onOutputOperationStarted(self, outputOperationStarted): | ||
""" | ||
Called when processing of a job of a batch has started. | ||
""" | ||
pass | ||
|
||
def onOutputOperationCompleted(self, outputOperationCompleted): | ||
""" | ||
Called when processing of a job of a batch has completed | ||
""" | ||
pass | ||
|
||
def getEventInfo(self, event): | ||
""" | ||
:param event: StreamingListenerEvent | ||
:return Returns a BatchInfo, OutputOperationInfo, or ReceiverInfo based on | ||
event passed. | ||
""" | ||
event_name = event.getClass().getSimpleName() | ||
if 'Batch' in event_name: | ||
return BatchInfo(event.batchInfo()) | ||
|
||
elif 'Output' in event_name: | ||
return OutputOperationInfo(event.outputOperationInfo()) | ||
|
||
elif 'Receiver' in event_name: | ||
return ReceiverInfo(event.receiverInfo()) | ||
|
||
class Java: | ||
implements = ["org.apache.spark.streaming.scheduler.StreamingListener"] | ||
|
||
|
||
class BatchInfo(object): | ||
|
||
def __init__(self, javaBatchInfo): | ||
|
||
self.processingStartTime = None | ||
self.processingEndTime = None | ||
|
||
self.batchTime = javaBatchInfo.batchTime() | ||
self.streamIdToInputInfo = self._map2dict(javaBatchInfo.streamIdToInputInfo()) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
|
||
self.submissionTime = javaBatchInfo.submissionTime() | ||
if javaBatchInfo.processingStartTime().isEmpty() is False: | ||
self.processingStartTime = javaBatchInfo.processingStartTime().get() | ||
if javaBatchInfo.processingEndTime().isEmpty() is False: | ||
self.processingEndTime = javaBatchInfo.processingEndTime().get() | ||
|
||
self.outputOperationInfos = self._map2dict(javaBatchInfo.outputOperationInfos()) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more.
|
||
|
||
def schedulingDelay(self): | ||
""" | ||
Time taken for the first job of this batch to start processing from the time this batch | ||
was submitted to the streaming scheduler. | ||
""" | ||
if self.processingStartTime is None: | ||
return None | ||
else: | ||
return self.processingStartTime - self.submissionTime | ||
|
||
def processingDelay(self): | ||
""" | ||
Time taken for the all jobs of this batch to finish processing from the time they started | ||
processing. | ||
""" | ||
if self.processingEndTime is None or self.processingStartTime is None: | ||
return None | ||
else: | ||
return self.processingEndTime - self.processingStartTime | ||
|
||
def totalDelay(self): | ||
""" | ||
Time taken for all the jobs of this batch to finish processing from the time they | ||
were submitted | ||
""" | ||
if self.processingEndTime is None or self.processingStartTime is None: | ||
return None | ||
else: | ||
return self.processingDelay() + self.schedulingDelay() | ||
|
||
def numRecords(self): | ||
""" | ||
The number of recorders received by the receivers in this batch. | ||
""" | ||
return len(self.streamIdToInputInfo) | ||
|
||
def _map2dict(self, javaMap): | ||
""" | ||
Converts a scala.collection.immutable.Map to a Python dict | ||
""" | ||
mapping = dict() | ||
map_iterator = javaMap.iterator() | ||
while map_iterator.hasNext(): | ||
entry = map_iterator.next() | ||
mapping[entry._1()] = entry._2() | ||
return mapping | ||
|
||
|
||
class OutputOperationInfo(object): | ||
|
||
def __init__(self, outputOperationInfo): | ||
self.batchTime = outputOperationInfo.batchTime() | ||
self.id = outputOperationInfo.id() | ||
self.name = outputOperationInfo.name() | ||
self.startTime = None | ||
if outputOperationInfo.startTime().isEmpty() is False: | ||
self.startTime = outputOperationInfo.startTime().get() | ||
self.endTime = None | ||
if outputOperationInfo.endTime().isEmpty() is False: | ||
self.endTime = outputOperationInfo.endTime().get() | ||
self.failureReason = None | ||
if outputOperationInfo.failureReason().isEmpty() is False: | ||
self.failureReason = outputOperationInfo.failureReason().get() | ||
|
||
def duration(self): | ||
if self.endTime is None or self.startTime is None: | ||
return None | ||
else: | ||
return self.endTime - self.startTime | ||
|
||
|
||
class ReceiverInfo(object): | ||
|
||
def __init__(self, receiverInfo): | ||
self.streamId = receiverInfo.streamId() | ||
self.name = receiverInfo.name() | ||
self.active = receiverInfo.active() | ||
self.location = receiverInfo.location() | ||
self.lastErrorMessage = receiverInfo.lastErrorMessage() | ||
self.lastError = receiverInfo.lastError() | ||
self.lastErrorTime = receiverInfo.lastErrorTime() |
Original file line number | Diff line number | Diff line change |
---|---|---|
|
@@ -43,6 +43,7 @@ | |
from pyspark.streaming.flume import FlumeUtils | ||
from pyspark.streaming.mqtt import MQTTUtils | ||
from pyspark.streaming.kinesis import KinesisUtils, InitialPositionInStream | ||
from pyspark.streaming.listener import StreamingListener, BatchInfo | ||
|
||
|
||
class PySparkStreamingTestCase(unittest.TestCase): | ||
|
@@ -398,6 +399,69 @@ def func(dstream): | |
self._test_func(input, func, expected) | ||
|
||
|
||
class StreamingListenerTests(PySparkStreamingTestCase): | ||
|
||
duration = .5 | ||
|
||
class BatchInfoCollector(StreamingListener): | ||
|
||
def __init__(self): | ||
super(StreamingListener, self).__init__() | ||
self.batchInfosCompleted = [] | ||
self.batchInfosStarted = [] | ||
self.batchInfosSubmitted = [] | ||
|
||
def onBatchSubmitted(self, batchSubmitted): | ||
self.batchInfosSubmitted.append(self.getEventInfo(batchSubmitted)) | ||
|
||
def onBatchStarted(self, batchStarted): | ||
self.batchInfosStarted.append(self.getEventInfo(batchStarted)) | ||
|
||
def onBatchCompleted(self, batchCompleted): | ||
self.batchInfosCompleted.append(self.getEventInfo(batchCompleted)) | ||
|
||
def test_batch_info_reports(self): | ||
batch_collector = self.BatchInfoCollector() | ||
self.ssc.addStreamingListener(batch_collector) | ||
input = [[1], [2], [3], [4]] | ||
|
||
def func(dstream): | ||
return dstream.map(int) | ||
expected = [[1], [2], [3], [4]] | ||
self._test_func(input, func, expected) | ||
|
||
# Test occasionally fails without a delay | ||
time.sleep(.1) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The listener sometimes receives 3 batchCompleted events instead of 4 when I run the test, even though it always gets the correct job output. However, when I add a minor delay the test passes consistently. Can I get someone's opinion on this? There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Take a look at |
||
|
||
batchInfosSubmitted = batch_collector.batchInfosSubmitted | ||
self.assertEqual(len(batchInfosSubmitted), 4) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. len(batchInfosSubmitted) may be greater than 4 since Streaming keeps launching batches. There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Ok, I'll change the assert to check if it receives at least 1 batch. |
||
|
||
for info in batchInfosSubmitted: | ||
|
||
self.assertIsNone(info.schedulingDelay()) | ||
self.assertIsNone(info.processingDelay()) | ||
self.assertIsNone(info.totalDelay()) | ||
|
||
batchInfosStarted = batch_collector.batchInfosStarted | ||
self.assertEqual(len(batchInfosStarted), 4) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. len(batchInfosSubmitted) may be greater than 4 since Streaming keeps launching batches. |
||
for info in batchInfosStarted: | ||
self.assertIsNotNone(info.schedulingDelay()) | ||
self.assertGreaterEqual(info.schedulingDelay(), 0) | ||
self.assertIsNone(info.processingDelay()) | ||
self.assertIsNone(info.totalDelay()) | ||
|
||
batchInfosCompleted = batch_collector.batchInfosCompleted | ||
self.assertEqual(len(batchInfosCompleted), 4) | ||
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. len(batchInfosSubmitted) may be greater than 4 since Streaming keeps launching batches. |
||
|
||
for info in batchInfosCompleted: | ||
self.assertIsNotNone(info.schedulingDelay()) | ||
self.assertIsNotNone(info.processingDelay()) | ||
self.assertIsNotNone(info.totalDelay()) | ||
self.assertGreaterEqual(info.schedulingDelay(), 0) | ||
self.assertGreaterEqual(info.processingDelay(), 0) | ||
self.assertGreaterEqual(info.totalDelay(), 0) | ||
|
||
|
||
class WindowFunctionTests(PySparkStreamingTestCase): | ||
|
||
timeout = 15 | ||
|
@@ -1281,7 +1345,8 @@ def search_kinesis_asl_assembly_jar(): | |
|
||
os.environ["PYSPARK_SUBMIT_ARGS"] = "--jars %s pyspark-shell" % jars | ||
testcases = [BasicOperationTests, WindowFunctionTests, StreamingContextTests, CheckpointTests, | ||
KafkaStreamTests, FlumeStreamTests, FlumePollingStreamTests, MQTTStreamTests] | ||
KafkaStreamTests, FlumeStreamTests, FlumePollingStreamTests, MQTTStreamTests, | ||
StreamingListenerTests] | ||
|
||
if kinesis_jar_present is True: | ||
testcases.append(KinesisStreamTests) | ||
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Is this method necessary?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I thought it would make it easier to see which Info is associated with each StreamingListenerEvent but I can remove it.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Would it be better to use this method to return instances of the Python friendly classes for these Scala objects?