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 @@ -402,7 +402,7 @@ object KafkaCluster {
}

Seq("zookeeper.connect", "group.id").foreach { s =>
if (!props.contains(s)) {
if (!props.containsKey(s)) {
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

That's a good bug fix

props.setProperty(s, "")
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,10 @@ class KafkaRDD[
.build()
val resp = consumer.fetch(req)
handleFetchErr(resp)

kc.setConsumerOffsets(kc.config.props.getString("group.id"),
Map(TopicAndPartition(part.topic, part.partition) ->
Math.min(kc.config.fetchMessageMaxBytes - part.fromOffset, part.untilOffset)))
// kafka may return a batch that starts before the requested offset
resp.messageSet(part.topic, part.partition)
.iterator
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@ import java.util.{Map => JMap}
import java.util.{Set => JSet}
import java.util.{List => JList}

import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset

import scala.reflect.ClassTag
import scala.collection.JavaConversions._

Expand Down Expand Up @@ -403,12 +405,23 @@ object KafkaUtils {
topicPartitions <- kc.getPartitions(topics).right
leaderOffsets <- (if (reset == Some("smallest")) {
kc.getEarliestLeaderOffsets(topicPartitions)
} else {
} else if (reset == Some("largest")){
kc.getLatestLeaderOffsets(topicPartitions)
} else {
val groupId = kafkaParams.get("group.id") match {
case Some(g) => g
case _ => throw new SparkException("need group.id in kafkaParams")
}
val offsets =
kc.getConsumerOffsets(kafkaParams.getOrElse("group.id", ""), topicPartitions.seq)
if(offsets.isLeft) kc.getLatestLeaderOffsets(topicPartitions)
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This style is pretty wrong

else offsets
}).right
} yield {
val fromOffsets = leaderOffsets.map { case (tp, lo) =>
(tp, lo.offset)
val fromOffsets = leaderOffsets.map {
case (tp, LeaderOffset(_, _, lo)) => (tp, lo)
case (tp, lo: Long) => (tp, lo)
case _ => throw new SparkException ("get error leader offsets ")
Copy link
Copy Markdown
Member

Choose a reason for hiding this comment

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

This is not a useful exception

}
new DirectKafkaInputDStream[K, V, KD, VD, (K, V)](
ssc, kafkaParams, fromOffsets, messageHandler)
Expand Down