Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
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 @@ -18,6 +18,9 @@
// scalastyle:off println
package org.apache.spark.examples.streaming

import org.apache.kafka.clients.consumer.ConsumerConfig
import org.apache.kafka.common.serialization.StringDeserializer

import org.apache.spark.SparkConf
import org.apache.spark.streaming._
import org.apache.spark.streaming.kafka010._
Expand All @@ -26,18 +29,20 @@ import org.apache.spark.streaming.kafka010._
* Consumes messages from one or more topics in Kafka and does wordcount.
* Usage: DirectKafkaWordCount <brokers> <topics>
* <brokers> is a list of one or more Kafka brokers
* <groupId> is a consumer group name to consume from topics
* <topics> is a list of one or more kafka topics to consume from
*
* Example:
* $ bin/run-example streaming.DirectKafkaWordCount broker1-host:port,broker2-host:port \
* topic1,topic2
* consumer-group topic1,topic2
*/
object DirectKafkaWordCount {
def main(args: Array[String]) {
if (args.length < 2) {
if (args.length < 3) {
System.err.println(s"""
|Usage: DirectKafkaWordCount <brokers> <topics>
| <brokers> is a list of one or more Kafka brokers
| <groupId> is a consumer group name to consume from topics
| <topics> is a list of one or more kafka topics to consume from
|
""".stripMargin)
Expand All @@ -46,15 +51,19 @@ object DirectKafkaWordCount {

StreamingExamples.setStreamingLogLevels()

val Array(brokers, topics) = args
val Array(brokers, groupId, topics) = args

// Create context with 2 second batch interval
val sparkConf = new SparkConf().setAppName("DirectKafkaWordCount")
val ssc = new StreamingContext(sparkConf, Seconds(2))

// Create direct kafka stream with brokers and topics
val topicsSet = topics.split(",").toSet
val kafkaParams = Map[String, String]("metadata.broker.list" -> brokers)
val kafkaParams = Map[String, Object](
ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> brokers,
ConsumerConfig.GROUP_ID_CONFIG -> groupId,
ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> classOf[StringDeserializer],
ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> classOf[StringDeserializer])
val messages = KafkaUtils.createDirectStream[String, String](
ssc,
LocationStrategies.PreferConsistent,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -190,8 +190,20 @@ private[spark] class DirectKafkaInputDStream[K, V](

// make sure new partitions are reflected in currentOffsets
val newPartitions = parts.diff(currentOffsets.keySet)

// Check if there's any partition been revoked because of consumer rebalance.
val revokedPartitions = currentOffsets.keySet.diff(parts)
if (revokedPartitions.nonEmpty) {
throw new IllegalStateException(s"Previously tracked partitions " +
s"${revokedPartitions.mkString("[", ",", "]")} been revoked by Kafka because of consumer " +
s"rebalance. This is mostly due to another stream with same group id joined, " +
s"please check if there're different streaming application misconfigure to use same " +
s"group id. Fundamentally different stream should use different group id")
}

// position for new partitions determined by auto.offset.reset if no commit
currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap

// don't want to consume messages, so pause
c.pause(newPartitions.asJava)
// find latest available offsets
Expand Down