Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
Show all changes
23 commits
Select commit Hold shift + click to select a range
e768164
#2808 update kafka to version 0.8.2
Dec 7, 2014
d9dc2bc
Merge remote-tracking branch 'upstream/master' into wip-2808-kafka-0.…
Dec 23, 2014
2e67c66
#SPARK-2808 Update to Kafka 0.8.2.0 GA from beta.
Feb 5, 2015
6953429
[SPARK-2808][Streaming][Kafka] update kafka to 0.8.2
koeninger Feb 11, 2015
77de6c2
Merge branch 'master' into wip-2808-kafka-0.8.2-upgrade
koeninger Mar 18, 2015
407382e
[SPARK-2808][Streaming][Kafka] update kafka to 0.8.2.1
koeninger Mar 18, 2015
ed02d2c
[SPARK-2808][Streaming][Kafka] move default argument for api version …
koeninger Apr 15, 2015
1d10751
Merge branch 'master' into wip-2808-kafka-0.8.2-upgrade
koeninger Apr 27, 2015
c70ee43
[SPARK-2808][Streaming][Kafka] add more asserts to test, try to figur…
koeninger Apr 28, 2015
9edab4c
[SPARK-2808][Streaming][Kafka] more shots in the dark on jenkins fail…
koeninger Apr 28, 2015
af6f3ec
[SPARK-2808][Streaming][Kafka] delay test until latest leader offset …
koeninger Apr 29, 2015
61b3464
[SPARK-2808][Streaming][Kafka] delay for second send in boundary cond…
koeninger Apr 29, 2015
3824ce3
[SPARK-2808][Streaming][Kafka] naming / comments per tdas
koeninger Apr 29, 2015
2b92d3f
[SPARK-2808][Streaming][Kafka] wait for leader offsets in the java te…
koeninger Apr 29, 2015
2712649
[SPARK-2808][Streaming][Kafka] add more logging to python test, see w…
koeninger Apr 29, 2015
115aeee
Merge branch 'master' into wip-2808-kafka-0.8.2-upgrade
koeninger Apr 29, 2015
4c4557f
[SPARK-2808][Streaming][Kafka] add even more logging to python test
koeninger Apr 30, 2015
1d896e2
[SPARK-2808][Streaming][Kafka] add even even more logging to python test
koeninger Apr 30, 2015
30d991d
[SPARK-2808][Streaming][Kafka] remove stderr prints since it breaks p…
koeninger May 1, 2015
d4267e9
[SPARK-2808][Streaming][Kafka] fix stderr redirect in python test script
koeninger May 1, 2015
1770abc
[SPARK-2808][Streaming][Kafka] make waitUntilLeaderOffset easier to c…
koeninger May 1, 2015
e6dfaf6
[SPARK-2808][Streaming][Kafka] pointless whitespace change to trigger…
koeninger May 1, 2015
803aa2c
[SPARK-2808][Streaming][Kafka] code cleanup per TD
koeninger May 1, 2015
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
Prev Previous commit
Next Next commit
[SPARK-2808][Streaming][Kafka] make waitUntilLeaderOffset easier to c…
…all, call it from python tests as well
  • Loading branch information
koeninger committed May 1, 2015
commit 1770abc64eaa8123301345f68e66aa2647bf2033
Original file line number Diff line number Diff line change
Expand Up @@ -231,11 +231,11 @@ private class KafkaTestUtils extends Logging {

/** wait until the leader offset for the given topic / partition equals the specified offset */
def waitUntilLeaderOffset(
Copy link
Contributor

Choose a reason for hiding this comment

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

What does this function do? Either make the name more meaningful (like waitUntilMetadataIsPropagated) or add scala docs.

kc: KafkaCluster,
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(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -72,9 +72,8 @@ public void testKafkaRDD() throws InterruptedException {
HashMap<String, String> kafkaParams = new HashMap<String, String>();
kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress());

KafkaCluster kc = new KafkaCluster(kafkaParams);
kafkaTestUtils.waitUntilLeaderOffset(kc, topic1, 0, topic1data.length);
kafkaTestUtils.waitUntilLeaderOffset(kc, topic2, 0, topic2data.length);
kafkaTestUtils.waitUntilLeaderOffset(topic1, 0, topic1data.length);
kafkaTestUtils.waitUntilLeaderOffset(topic2, 0, topic2data.length);

OffsetRange[] offsetRanges = {
OffsetRange.create(topic1, 0, 0, 1),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,8 +61,7 @@ class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll {
val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress,
"group.id" -> s"test-consumer-${Random.nextInt}")

val kc = new KafkaCluster(kafkaParams)
kafkaTestUtils.waitUntilLeaderOffset(kc, topic, 0, messages.size)
kafkaTestUtils.waitUntilLeaderOffset(topic, 0, messages.size)

val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size))

Expand All @@ -87,7 +86,7 @@ class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll {
// this is the "lots of messages" case
kafkaTestUtils.sendMessages(topic, sent)
val sentCount = sent.values.sum
kafkaTestUtils.waitUntilLeaderOffset(kc, topic, 0, sentCount)
kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount)

// rdd defined from leaders after sending messages, should get the number sent
val rdd = getRdd(kc, Set(topic))
Expand All @@ -113,7 +112,7 @@ class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll {
val sentOnlyOne = Map("d" -> 1)

kafkaTestUtils.sendMessages(topic, sentOnlyOne)
kafkaTestUtils.waitUntilLeaderOffset(kc, topic, 0, sentCount + 1)
kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sentCount + 1)

assert(rdd2.isDefined)
assert(rdd2.get.count === 0, "got messages when there shouldn't be any")
Expand Down
6 changes: 3 additions & 3 deletions python/pyspark/streaming/tests.py
Original file line number Diff line number Diff line change
Expand Up @@ -615,7 +615,7 @@ def test_kafka_stream(self):

self._kafkaTestUtils.createTopic(topic)
self._kafkaTestUtils.sendMessages(topic, sendData)
time.sleep(5)
self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values()))

stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(),
"test-streaming-consumer", {topic: 1},
Expand Down Expand Up @@ -659,7 +659,7 @@ def test_kafka_rdd(self):

self._kafkaTestUtils.createTopic(topic)
self._kafkaTestUtils.sendMessages(topic, sendData)
time.sleep(5)
self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values()))
Copy link
Contributor

Choose a reason for hiding this comment

The 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 test_kafka_direct_stream***

rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges)
self._validateRddResult(sendData, rdd)

Expand All @@ -675,7 +675,7 @@ def test_kafka_rdd_with_leaders(self):

self._kafkaTestUtils.createTopic(topic)
self._kafkaTestUtils.sendMessages(topic, sendData)
time.sleep(5)
self._kafkaTestUtils.waitUntilLeaderOffset(topic, 0, sum(sendData.values()))
rdd = KafkaUtils.createRDD(self.sc, kafkaParams, offsetRanges, leaders)
self._validateRddResult(sendData, rdd)

Expand Down