-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-6766][Streaming] Fix issue about StreamingListenerBatchSubmitted and StreamingListenerBatchStarted #5414
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
Closed
Closed
Changes from all commits
Commits
Show all changes
6 commits
Select commit
Hold shift + click to select a range
493f978
Send StreamingListenerBatchSubmitted when JobSet is submitted; fix St…
zsxwing 74aed99
Refactor as per TD's suggestion
zsxwing fc3a2a1
Add unit tests for SPARK-6766
zsxwing 79b4fed
Add StreamingJobProgressListenerSuite to test StreamingJobProgressLis…
zsxwing ca0955b
Combine unit tests
zsxwing 2f85060
Update tests
zsxwing File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -46,10 +46,38 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { | |
| val collector = new BatchInfoCollector | ||
| ssc.addStreamingListener(collector) | ||
| runStreams(ssc, input.size, input.size) | ||
| val batchInfos = collector.batchInfos | ||
| batchInfos should have size 4 | ||
|
|
||
| batchInfos.foreach(info => { | ||
| // SPARK-6766: batch info should be submitted | ||
| val batchInfosSubmitted = collector.batchInfosSubmitted | ||
| batchInfosSubmitted should have size 4 | ||
|
Contributor
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. Nit: these tests for submitted and started should come before completed :) |
||
|
|
||
| batchInfosSubmitted.foreach(info => { | ||
| info.schedulingDelay should be (None) | ||
| info.processingDelay should be (None) | ||
| info.totalDelay should be (None) | ||
| }) | ||
|
|
||
| isInIncreasingOrder(batchInfosSubmitted.map(_.submissionTime)) should be (true) | ||
|
|
||
| // SPARK-6766: processingStartTime of batch info should not be None when starting | ||
| val batchInfosStarted = collector.batchInfosStarted | ||
| batchInfosStarted should have size 4 | ||
|
|
||
| batchInfosStarted.foreach(info => { | ||
| info.schedulingDelay should not be None | ||
| info.schedulingDelay.get should be >= 0L | ||
| info.processingDelay should be (None) | ||
| info.totalDelay should be (None) | ||
| }) | ||
|
|
||
| isInIncreasingOrder(batchInfosStarted.map(_.submissionTime)) should be (true) | ||
| isInIncreasingOrder(batchInfosStarted.map(_.processingStartTime.get)) should be (true) | ||
|
|
||
| // test onBatchCompleted | ||
| val batchInfosCompleted = collector.batchInfosCompleted | ||
| batchInfosCompleted should have size 4 | ||
|
|
||
| batchInfosCompleted.foreach(info => { | ||
| info.schedulingDelay should not be None | ||
| info.processingDelay should not be None | ||
| info.totalDelay should not be None | ||
|
|
@@ -58,9 +86,9 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { | |
| info.totalDelay.get should be >= 0L | ||
| }) | ||
|
|
||
| isInIncreasingOrder(batchInfos.map(_.submissionTime)) should be (true) | ||
| isInIncreasingOrder(batchInfos.map(_.processingStartTime.get)) should be (true) | ||
| isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) | ||
| isInIncreasingOrder(batchInfosCompleted.map(_.submissionTime)) should be (true) | ||
| isInIncreasingOrder(batchInfosCompleted.map(_.processingStartTime.get)) should be (true) | ||
| isInIncreasingOrder(batchInfosCompleted.map(_.processingEndTime.get)) should be (true) | ||
| } | ||
|
|
||
| test("receiver info reporting") { | ||
|
|
@@ -99,9 +127,20 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { | |
|
|
||
| /** Listener that collects information on processed batches */ | ||
| class BatchInfoCollector extends StreamingListener { | ||
| val batchInfos = new ArrayBuffer[BatchInfo] | ||
| val batchInfosCompleted = new ArrayBuffer[BatchInfo] | ||
| val batchInfosStarted = new ArrayBuffer[BatchInfo] | ||
| val batchInfosSubmitted = new ArrayBuffer[BatchInfo] | ||
|
|
||
| override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { | ||
| batchInfosSubmitted += batchSubmitted.batchInfo | ||
| } | ||
|
|
||
| override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { | ||
| batchInfosStarted += batchStarted.batchInfo | ||
| } | ||
|
|
||
| override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { | ||
| batchInfos += batchCompleted.batchInfo | ||
| batchInfosCompleted += batchCompleted.batchInfo | ||
| } | ||
| } | ||
|
|
||
|
|
||
119 changes: 119 additions & 0 deletions
119
...ming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,119 @@ | ||
| /* | ||
| * 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. | ||
| */ | ||
|
|
||
| package org.apache.spark.streaming.ui | ||
|
|
||
| import org.scalatest.Matchers | ||
|
|
||
| import org.apache.spark.streaming.dstream.DStream | ||
| import org.apache.spark.streaming.scheduler._ | ||
| import org.apache.spark.streaming.{Time, Milliseconds, TestSuiteBase} | ||
|
|
||
| class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { | ||
|
|
||
| val input = (1 to 4).map(Seq(_)).toSeq | ||
| val operation = (d: DStream[Int]) => d.map(x => x) | ||
|
|
||
| override def batchDuration = Milliseconds(100) | ||
|
|
||
| test("onBatchSubmitted, onBatchStarted, onBatchCompleted, " + | ||
| "onReceiverStarted, onReceiverError, onReceiverStopped") { | ||
| val ssc = setupStreams(input, operation) | ||
| val listener = new StreamingJobProgressListener(ssc) | ||
|
|
||
| val receivedBlockInfo = Map( | ||
| 0 -> Array(ReceivedBlockInfo(0, 100, null), ReceivedBlockInfo(0, 200, null)), | ||
| 1 -> Array(ReceivedBlockInfo(1, 300, null)) | ||
| ) | ||
|
|
||
| // onBatchSubmitted | ||
| val batchInfoSubmitted = BatchInfo(Time(1000), receivedBlockInfo, 1000, None, None) | ||
| listener.onBatchSubmitted(StreamingListenerBatchSubmitted(batchInfoSubmitted)) | ||
| listener.waitingBatches should be (List(batchInfoSubmitted)) | ||
| listener.runningBatches should be (Nil) | ||
| listener.retainedCompletedBatches should be (Nil) | ||
| listener.lastCompletedBatch should be (None) | ||
| listener.numUnprocessedBatches should be (1) | ||
| listener.numTotalCompletedBatches should be (0) | ||
| listener.numTotalProcessedRecords should be (0) | ||
| listener.numTotalReceivedRecords should be (0) | ||
|
|
||
| // onBatchStarted | ||
| val batchInfoStarted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) | ||
| listener.onBatchStarted(StreamingListenerBatchStarted(batchInfoStarted)) | ||
| listener.waitingBatches should be (Nil) | ||
| listener.runningBatches should be (List(batchInfoStarted)) | ||
| listener.retainedCompletedBatches should be (Nil) | ||
| listener.lastCompletedBatch should be (None) | ||
| listener.numUnprocessedBatches should be (1) | ||
| listener.numTotalCompletedBatches should be (0) | ||
| listener.numTotalProcessedRecords should be (0) | ||
| listener.numTotalReceivedRecords should be (600) | ||
|
|
||
| // onBatchCompleted | ||
| val batchInfoCompleted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) | ||
| listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) | ||
| listener.waitingBatches should be (Nil) | ||
| listener.runningBatches should be (Nil) | ||
| listener.retainedCompletedBatches should be (List(batchInfoCompleted)) | ||
| listener.lastCompletedBatch should be (Some(batchInfoCompleted)) | ||
| listener.numUnprocessedBatches should be (0) | ||
| listener.numTotalCompletedBatches should be (1) | ||
| listener.numTotalProcessedRecords should be (600) | ||
| listener.numTotalReceivedRecords should be (600) | ||
|
|
||
| // onReceiverStarted | ||
| val receiverInfoStarted = ReceiverInfo(0, "test", null, true, "localhost") | ||
| listener.onReceiverStarted(StreamingListenerReceiverStarted(receiverInfoStarted)) | ||
| listener.receiverInfo(0) should be (Some(receiverInfoStarted)) | ||
| listener.receiverInfo(1) should be (None) | ||
|
|
||
| // onReceiverError | ||
| val receiverInfoError = ReceiverInfo(1, "test", null, true, "localhost") | ||
| listener.onReceiverError(StreamingListenerReceiverError(receiverInfoError)) | ||
| listener.receiverInfo(0) should be (Some(receiverInfoStarted)) | ||
| listener.receiverInfo(1) should be (Some(receiverInfoError)) | ||
| listener.receiverInfo(2) should be (None) | ||
|
|
||
| // onReceiverStopped | ||
| val receiverInfoStopped = ReceiverInfo(2, "test", null, true, "localhost") | ||
| listener.onReceiverStopped(StreamingListenerReceiverStopped(receiverInfoStopped)) | ||
| listener.receiverInfo(0) should be (Some(receiverInfoStarted)) | ||
| listener.receiverInfo(1) should be (Some(receiverInfoError)) | ||
| listener.receiverInfo(2) should be (Some(receiverInfoStopped)) | ||
| listener.receiverInfo(3) should be (None) | ||
| } | ||
|
|
||
| test("Remove the old completed batches when exceeding the limit") { | ||
| val ssc = setupStreams(input, operation) | ||
| val limit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) | ||
| val listener = new StreamingJobProgressListener(ssc) | ||
|
|
||
| val receivedBlockInfo = Map( | ||
| 0 -> Array(ReceivedBlockInfo(0, 100, null), ReceivedBlockInfo(0, 200, null)), | ||
| 1 -> Array(ReceivedBlockInfo(1, 300, null)) | ||
| ) | ||
| val batchInfoCompleted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) | ||
|
|
||
| for(_ <- 0 until (limit + 10)) { | ||
| listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) | ||
| } | ||
|
|
||
| listener.retainedCompletedBatches.size should be (limit) | ||
| listener.numTotalCompletedBatches should be(limit + 10) | ||
| } | ||
| } |
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
Good catch. OMG!