Skip to content
Snippets Groups Projects
Commit 14502d5e authored by zsxwing's avatar zsxwing Committed by Tathagata Das
Browse files

[SPARK-7405] [STREAMING] Fix the bug that ReceiverInputDStream doesn't report InputInfo

The bug is because SPARK-7139 removed some codes from SPARK-7112 unintentionally here: https://github.com/apache/spark/commit/1854ac326a9cc6014817d8df30ed0458eee5d7d1#diff-5c8651dd78abd20439b8eb938175075dL72

This PR just added them back and added some assertions in the tests to verify it.

Author: zsxwing <zsxwing@gmail.com>

Closes #5950 from zsxwing/SPARK-7405 and squashes the following commits:

675f5d9 [zsxwing] Fix the bug that ReceiverInputDStream doesn't report InputInfo
parent 71a452b6
No related branches found
No related tags found
No related merge requests found
......@@ -24,6 +24,7 @@ import org.apache.spark.storage.BlockId
import org.apache.spark.streaming._
import org.apache.spark.streaming.rdd.WriteAheadLogBackedBlockRDD
import org.apache.spark.streaming.receiver.Receiver
import org.apache.spark.streaming.scheduler.InputInfo
import org.apache.spark.streaming.util.WriteAheadLogUtils
/**
......@@ -68,6 +69,10 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont
val blockInfos = receiverTracker.getBlocksOfBatch(validTime).getOrElse(id, Seq.empty)
val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray
// Register the input blocks information into InputInfoTracker
val inputInfo = InputInfo(id, blockInfos.map(_.numRecords).sum)
ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo)
// Are WAL record handles present with all the blocks
val areWALRecordHandlesPresent = blockInfos.forall { _.walRecordHandleOption.nonEmpty }
......
......@@ -50,6 +50,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
// Set up the streaming context and input streams
withStreamingContext(new StreamingContext(conf, batchDuration)) { ssc =>
ssc.addStreamingListener(ssc.progressListener)
val input = Seq(1, 2, 3, 4, 5)
// Use "batchCount" to make sure we check the result after all batches finish
val batchCounter = new BatchCounter(ssc)
......@@ -72,6 +74,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter {
if (!batchCounter.waitUntilBatchesCompleted(input.size, 30000)) {
fail("Timeout: cannot finish all batches in 30 seconds")
}
// Verify all "InputInfo"s have been reported
assert(ssc.progressListener.numTotalReceivedRecords === input.size)
assert(ssc.progressListener.numTotalProcessedRecords === input.size)
logInfo("Stopping server")
testServer.stop()
logInfo("Stopping context")
......
0% Loading or .
You are about to add 0 people to the discussion. Proceed with caution.
Finish editing this message first!
Please register or to comment