Skip to content

Commit 200c980

Browse files
zsxwingtdas
authored andcommitted
[SPARK-8112] [STREAMING] Fix the negative event count issue
Author: zsxwing <zsxwing@gmail.com> Closes #6659 from zsxwing/SPARK-8112 and squashes the following commits: a5d7da6 [zsxwing] Address comments d255b6e [zsxwing] Fix the negative event count issue (cherry picked from commit 4f16d3f) Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
1 parent 429c658 commit 200c980

File tree

5 files changed

+10
-6
lines changed

5 files changed

+10
-6
lines changed

streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -70,7 +70,7 @@ abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingCont
7070
val blockIds = blockInfos.map { _.blockId.asInstanceOf[BlockId] }.toArray
7171

7272
// Register the input blocks information into InputInfoTracker
73-
val inputInfo = InputInfo(id, blockInfos.map(_.numRecords).sum)
73+
val inputInfo = InputInfo(id, blockInfos.flatMap(_.numRecords).sum)
7474
ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo)
7575

7676
if (blockInfos.nonEmpty) {

streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -138,8 +138,8 @@ private[streaming] class ReceiverSupervisorImpl(
138138
) {
139139
val blockId = blockIdOption.getOrElse(nextBlockId)
140140
val numRecords = receivedBlock match {
141-
case ArrayBufferBlock(arrayBuffer) => arrayBuffer.size
142-
case _ => -1
141+
case ArrayBufferBlock(arrayBuffer) => Some(arrayBuffer.size.toLong)
142+
case _ => None
143143
}
144144

145145
val time = System.currentTimeMillis

streaming/src/main/scala/org/apache/spark/streaming/scheduler/InputInfoTracker.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,9 @@ import org.apache.spark.Logging
2323
import org.apache.spark.streaming.{Time, StreamingContext}
2424

2525
/** To track the information of input stream at specified batch time. */
26-
private[streaming] case class InputInfo(inputStreamId: Int, numRecords: Long)
26+
private[streaming] case class InputInfo(inputStreamId: Int, numRecords: Long) {
27+
require(numRecords >= 0, "numRecords must not be negative")
28+
}
2729

2830
/**
2931
* This class manages all the input streams as well as their input data statistics. The information

streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockInfo.scala

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -24,11 +24,13 @@ import org.apache.spark.streaming.util.WriteAheadLogRecordHandle
2424
/** Information about blocks received by the receiver */
2525
private[streaming] case class ReceivedBlockInfo(
2626
streamId: Int,
27-
numRecords: Long,
27+
numRecords: Option[Long],
2828
metadataOption: Option[Any],
2929
blockStoreResult: ReceivedBlockStoreResult
3030
) {
3131

32+
require(numRecords.isEmpty || numRecords.get >= 0, "numRecords must not be negative")
33+
3234
@volatile private var _isBlockIdValid = true
3335

3436
def blockId: StreamBlockId = blockStoreResult.blockId

streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -224,7 +224,7 @@ class ReceivedBlockTrackerSuite
224224

225225
/** Generate blocks infos using random ids */
226226
def generateBlockInfos(): Seq[ReceivedBlockInfo] = {
227-
List.fill(5)(ReceivedBlockInfo(streamId, 0, None,
227+
List.fill(5)(ReceivedBlockInfo(streamId, Some(0L), None,
228228
BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt)))))
229229
}
230230

0 commit comments

Comments
 (0)