-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-2808][Streaming][Kafka] update kafka to 0.8.2 #4537
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
e768164
d9dc2bc
2e67c66
6953429
77de6c2
407382e
ed02d2c
1d10751
c70ee43
9edab4c
af6f3ec
61b3464
3824ce3
2b92d3f
2712649
115aeee
4c4557f
1d896e2
30d991d
d4267e9
1770abc
e6dfaf6
803aa2c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -29,10 +29,12 @@ import scala.language.postfixOps | |
| import scala.util.control.NonFatal | ||
|
|
||
| import kafka.admin.AdminUtils | ||
| import kafka.api.Request | ||
| import kafka.common.TopicAndPartition | ||
| import kafka.producer.{KeyedMessage, Producer, ProducerConfig} | ||
| import kafka.serializer.StringEncoder | ||
| import kafka.server.{KafkaConfig, KafkaServer} | ||
| import kafka.utils.ZKStringSerializer | ||
| import kafka.utils.{ZKStringSerializer, ZkUtils} | ||
| import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} | ||
| import org.I0Itec.zkclient.ZkClient | ||
|
|
||
|
|
@@ -227,12 +229,35 @@ private class KafkaTestUtils extends Logging { | |
| tryAgain(1) | ||
| } | ||
|
|
||
| private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { | ||
| /** Wait until the leader offset for the given topic/partition equals the specified offset */ | ||
| def waitUntilLeaderOffset( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What does this function do? Either make the name more meaningful (like |
||
| topic: String, | ||
| partition: Int, | ||
| offset: Long): Unit = { | ||
| eventually(Time(10000), Time(100)) { | ||
| val kc = new KafkaCluster(Map("metadata.broker.list" -> brokerAddress)) | ||
| val tp = TopicAndPartition(topic, partition) | ||
| val llo = kc.getLatestLeaderOffsets(Set(tp)).right.get.apply(tp).offset | ||
| assert( | ||
| server.apis.metadataCache.containsTopicAndPartition(topic, partition), | ||
| s"Partition [$topic, $partition] metadata not propagated after timeout" | ||
| ) | ||
| llo == offset, | ||
| s"$topic $partition $offset not reached after timeout") | ||
| } | ||
| } | ||
|
|
||
| private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { | ||
| def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { | ||
| case Some(partitionState) => | ||
| val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr | ||
|
|
||
| ZkUtils.getLeaderForPartition(zkClient, topic, partition).isDefined && | ||
| Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && | ||
| leaderAndInSyncReplicas.isr.size >= 1 | ||
|
|
||
| case _ => | ||
| false | ||
| } | ||
| eventually(Time(10000), Time(100)) { | ||
| assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") | ||
| } | ||
| } | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -53,14 +53,15 @@ class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll { | |
| } | ||
|
|
||
| test("basic usage") { | ||
| val topic = "topicbasic" | ||
| val topic = s"topicbasic-${Random.nextInt}" | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Why was this change necessary? |
||
| kafkaTestUtils.createTopic(topic) | ||
| val messages = Set("the", "quick", "brown", "fox") | ||
| kafkaTestUtils.sendMessages(topic, messages.toArray) | ||
|
|
||
|
|
||
| val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, | ||
| "group.id" -> s"test-consumer-${Random.nextInt(10000)}") | ||
| "group.id" -> s"test-consumer-${Random.nextInt}") | ||
|
|
||
| kafkaTestUtils.waitUntilLeaderOffset(topic, 0, messages.size) | ||
|
|
||
| val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) | ||
|
|
||
|
|
@@ -73,34 +74,47 @@ class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll { | |
|
|
||
| test("iterator boundary conditions") { | ||
| // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd | ||
| val topic = "topic1" | ||
| val topic = s"topicboundary-${Random.nextInt}" | ||
| val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) | ||
| kafkaTestUtils.createTopic(topic) | ||
|
|
||
| val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, | ||
| "group.id" -> s"test-consumer-${Random.nextInt(10000)}") | ||
| "group.id" -> s"test-consumer-${Random.nextInt}") | ||
|
|
||
| val kc = new KafkaCluster(kafkaParams) | ||
|
|
||
| // this is the "lots of messages" case | ||
| kafkaTestUtils.sendMessages(topic, sent) | ||
| val sentCount = sent.values.sum | ||
| kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount) | ||
|
|
||
| // rdd defined from leaders after sending messages, should get the number sent | ||
| val rdd = getRdd(kc, Set(topic)) | ||
|
|
||
| assert(rdd.isDefined) | ||
| assert(rdd.get.count === sent.values.sum, "didn't get all sent messages") | ||
|
|
||
| val ranges = rdd.get.asInstanceOf[HasOffsetRanges] | ||
| .offsetRanges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap | ||
| val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges | ||
| val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum | ||
|
|
||
| kc.setConsumerOffsets(kafkaParams("group.id"), ranges) | ||
| assert(rangeCount === sentCount, "offset range didn't include all sent messages") | ||
| assert(rdd.get.count === sentCount, "didn't get all sent messages") | ||
|
|
||
| val rangesMap = ranges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap | ||
|
|
||
| // make sure consumer offsets are committed before the next getRdd call | ||
| kc.setConsumerOffsets(kafkaParams("group.id"), rangesMap).fold( | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. What is this for? Its not obvious, could you add a comment? |
||
| err => throw new Exception(err.mkString("\n")), | ||
| _ => () | ||
| ) | ||
|
|
||
| // this is the "0 messages" case | ||
| val rdd2 = getRdd(kc, Set(topic)) | ||
| // shouldn't get anything, since message is sent after rdd was defined | ||
| val sentOnlyOne = Map("d" -> 1) | ||
|
|
||
| kafkaTestUtils.sendMessages(topic, sentOnlyOne) | ||
| kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount + 1) | ||
|
|
||
| assert(rdd2.isDefined) | ||
| assert(rdd2.get.count === 0, "got messages when there shouldn't be any") | ||
|
|
||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -606,7 +606,6 @@ def _validateRddResult(self, sendData, rdd): | |
| result = {} | ||
| for i in rdd.map(lambda x: x[1]).collect(): | ||
| result[i] = result.get(i, 0) + 1 | ||
|
|
||
| self.assertEqual(sendData, result) | ||
|
|
||
| def test_kafka_stream(self): | ||
|
|
@@ -616,6 +615,7 @@ def test_kafka_stream(self): | |
|
|
||
| self._kafkaTestUtils.createTopic(topic) | ||
| self._kafkaTestUtils.sendMessages(topic, sendData) | ||
| self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) | ||
|
|
||
| stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(), | ||
| "test-streaming-consumer", {topic: 1}, | ||
|
|
@@ -631,6 +631,7 @@ def test_kafka_direct_stream(self): | |
|
|
||
| self._kafkaTestUtils.createTopic(topic) | ||
| self._kafkaTestUtils.sendMessages(topic, sendData) | ||
| self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) | ||
|
|
||
| stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams) | ||
| self._validateStreamResult(sendData, stream) | ||
|
|
@@ -645,6 +646,7 @@ def test_kafka_direct_stream_from_offset(self): | |
|
|
||
| self._kafkaTestUtils.createTopic(topic) | ||
| self._kafkaTestUtils.sendMessages(topic, sendData) | ||
| self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) | ||
|
|
||
| stream = KafkaUtils.createDirectStream(self.ssc, [topic], kafkaParams, fromOffsets) | ||
| self._validateStreamResult(sendData, stream) | ||
|
|
@@ -659,7 +661,7 @@ def test_kafka_rdd(self): | |
|
|
||
| self._kafkaTestUtils.createTopic(topic) | ||
| self._kafkaTestUtils.sendMessages(topic, sendData) | ||
|
|
||
| self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) | ||
|
Contributor
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. Should be added for the all the kafka unit tests, whereever there is a sendMessage. Its missing in |
||
| rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges) | ||
| self._validateRddResult(sendData, rdd) | ||
|
|
||
|
|
@@ -675,7 +677,7 @@ def test_kafka_rdd_with_leaders(self): | |
|
|
||
| self._kafkaTestUtils.createTopic(topic) | ||
| self._kafkaTestUtils.sendMessages(topic, sendData) | ||
|
|
||
| self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values())) | ||
| rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders) | ||
| self._validateRddResult(sendData, rdd) | ||
|
|
||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: These two can be merged with default arguments.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
The code was originally using default arguments. That's what was causing the MiMa binary compatibility errors
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
that is frigging weird. This is in an internal class and should not be throwing MIMA compatibility errors. :/
anyways, its fine as is. better than fighting with mima right now.