Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@

package org.apache.spark.streaming.kafka


import scala.annotation.tailrec
import scala.collection.mutable
import scala.reflect.{classTag, ClassTag}
Expand All @@ -27,10 +26,10 @@ import kafka.message.MessageAndMetadata
import kafka.serializer.Decoder

import org.apache.spark.{Logging, SparkException}
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
import org.apache.spark.streaming.{StreamingContext, Time}
import org.apache.spark.streaming.dstream._
import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset
import org.apache.spark.streaming.scheduler.InputInfo

/**
* A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where
Expand Down Expand Up @@ -117,6 +116,12 @@ class DirectKafkaInputDStream[
val rdd = KafkaRDD[K, V, U, T, R](
context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler)

// Report the number of records of the batch interval to InputInfoTracker.
val currentNumRecords = currentOffsets.map(_._2).sum
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

currentNumRecords sounds like "current number of records to be processed". That is not what this is. Also the way you are calculating this is not very intuitive. A better way would be

rdd.offsetRanges.map(range => range.untilOffset - range.fromOffset).sum

That is, map each range to the number of record and add them.

val toBeProcessedNumRecords = untilOffsets.map(_._2.offset).sum
val inputInfo = InputInfo(id, toBeProcessedNumRecords - currentNumRecords)
ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo)

currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset)
Some(rdd)
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ import org.apache.spark.{Logging, SparkConf, SparkContext}
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time}
import org.apache.spark.streaming.dstream.DStream
import org.apache.spark.streaming.scheduler._
import org.apache.spark.util.Utils

class DirectKafkaStreamSuite
Expand Down Expand Up @@ -290,7 +291,6 @@ class DirectKafkaStreamSuite
},
"Recovered ranges are not the same as the ones generated"
)

// Restart context, give more data and verify the total at the end
// If the total is write that means each records has been received only once
ssc.start()
Expand All @@ -301,6 +301,49 @@ class DirectKafkaStreamSuite
ssc.stop()
}

test("Direct Kafka stream report input information") {
val topic = "report-test"
val data = Map("a" -> 7, "b" -> 9)
kafkaTestUtils.createTopic(topic)
kafkaTestUtils.sendMessages(topic, data)

val totalSent = data.values.sum
val kafkaParams = Map(
"metadata.broker.list" -> kafkaTestUtils.brokerAddress,
"auto.offset.reset" -> "smallest"
)

import DirectKafkaStreamSuite._
ssc = new StreamingContext(sparkConf, Milliseconds(200))
val collector = new InputInfoCollector
ssc.addStreamingListener(collector)

val stream = withClue("Error creating direct stream") {
KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder](
ssc, kafkaParams, Set(topic))
}

val allReceived = new ArrayBuffer[(String, String)]

stream.foreachRDD { rdd => allReceived ++= rdd.collect() }
ssc.start()
eventually(timeout(20000.milliseconds), interval(200.milliseconds)) {
assert(allReceived.size === totalSent,
"didn't get expected number of messages, messages:\n" + allReceived.mkString("\n"))
}
ssc.stop()

// Calculate all the record number collected in the StreamingListener.
val numRecordsSubmitted = collector.streamIdToNumRecordsSubmitted.map(_.values.sum).sum
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is probably race condition here. The collector may not have received the batch completed signal when the allReceived.size == totalSent is satisfied and the context is stopped. Better to put all of these asserts under the eventually.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Also used AtomicLong or AtomicInt to handle multithread access. Otherwise there can be race conditions leading to flakiness.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I dont think there's a multi-thread issue, I tested the number of records until the StreamingContext is stopped, so I think at that point there's no other thread will access collector object.

Anyway I just only test total number of records, so AtomicLong is enough, I will change to that way.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

There is probably race condition here. The collector may not have received the batch completed signal when the allReceived.size == totalSent is satisfied and the context is stopped. Better to put all of these asserts under the eventually.

Not sure why all the assert should be in the eventually, from my understanding it is OK the last signal is missed, since we only test the total number of completed records.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

  1. JVM does not guarantee different threads will see the same values within any bounded period of time until some kind of synchronization is used. Has caused flakiness in the past.
  2. The StreamingListener events are sent on an async thread. So there is a time gap between when the last job finishes and the posting of the StreamingListenerBatchCompleted event. In the current code, the system may satisfy the eventually and stop the streamingContext before the event is dispatched and InputInfoCollector. onBatchCompleted() is called. In which case, things will fail. This will be fine probably 99.99% of the time. But on a place like Jenkins, that 0.01% chance causes annoying flakiness.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

OK, I get it, thanks a lot for your explanation.

assert(numRecordsSubmitted === totalSent)

val numRecordsStarted = collector.streamIdToNumRecordsStarted.map(_.values.sum).sum
assert(numRecordsStarted === totalSent)

val numRecordsCompleted = collector.streamIdToNumRecordsCompleted.map(_.values.sum).sum
assert(numRecordsCompleted === totalSent)
}

/** Get the generated offset ranges from the DirectKafkaStream */
private def getOffsetRanges[K, V](
kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = {
Expand All @@ -313,4 +356,24 @@ class DirectKafkaStreamSuite
object DirectKafkaStreamSuite {
val collectedData = new mutable.ArrayBuffer[String]()
var total = -1L

class InputInfoCollector extends StreamingListener {
val streamIdToNumRecordsSubmitted = new ArrayBuffer[Map[Int, Long]]()
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If you are just going to test the total number of records received, why do you need arraybuffers?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, that's not necessary, I will change it.

val streamIdToNumRecordsStarted = new ArrayBuffer[Map[Int, Long]]()
val streamIdToNumRecordsCompleted = new ArrayBuffer[Map[Int, Long]]()

override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit =
synchronized {
streamIdToNumRecordsSubmitted += batchSubmitted.batchInfo.streamIdToNumRecords
}

override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = synchronized {
streamIdToNumRecordsStarted += batchStarted.batchInfo.streamIdToNumRecords
}

override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit =
synchronized {
streamIdToNumRecordsCompleted += batchCompleted.batchInfo.streamIdToNumRecords
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -192,8 +192,8 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext)
val latestReceiverNumRecords = latestBatchInfos.map(_.receiverNumRecords)
val streamIds = ssc.graph.getInputStreams().map(_.id)
streamIds.map { id =>
val recordsOfParticularReceiver =
latestReceiverNumRecords.map(v => v.getOrElse(id, 0L).toDouble * 1000 / batchDuration)
val recordsOfParticularReceiver =
Copy link
Member

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You don't need to change this indent. This method will be removed in #5533

latestReceiverNumRecords.map(v => v.getOrElse(id, 0L).toDouble * 1000 / batchDuration)
val distribution = Distribution(recordsOfParticularReceiver)
(id, distribution)
}.toMap
Expand Down