-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-4964][Streaming][Kafka] More updates to Exactly-once Kafka stream #4384
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
Closed
Closed
Changes from 2 commits
Commits
Show all changes
10 commits
Select commit
Hold shift + click to select a range
6a91cab
Added example
tdas 4986784
Added unit test to kafka offset recovery
tdas e73589c
Minor changes.
tdas 50f2b56
Added Java API and added more Scala and Java unit tests. Also updated…
tdas bb65232
Fixed test bug and refactored KafkaStreamSuite
tdas e4abf69
Scala doc improvements and stuff.
tdas 26df23c
Updates based on PR comments from Cody
tdas 83d0402
Added JavaDirectKafkaWordCount example.
tdas 3ed9284
updated scala doc
tdas 7c931c3
Small update
tdas File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
There are no files selected for viewing
60 changes: 60 additions & 0 deletions
60
.../scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,60 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.examples.streaming | ||
|
|
||
| import kafka.serializer.StringDecoder | ||
|
|
||
| import org.apache.spark.streaming._ | ||
| import org.apache.spark.streaming.kafka._ | ||
| import org.apache.spark.SparkConf | ||
|
|
||
| /** | ||
| * Consumes messages from one or more topics in Kafka and does wordcount. | ||
| * Usage: DirectKafkaWordCount <brokers> <topics> | ||
| * <brokers> is a list of one or more zookeeper servers that make quorum | ||
| * <topics> is a list of one or more kafka topics to consume from | ||
| * | ||
| * Example: | ||
| * $ bin/run-example streaming.KafkaWordCount broker1-host:port,broker2-host:port topic1,topic2 | ||
| */ | ||
| object DirectKafkaWordCount { | ||
| def main(args: Array[String]) { | ||
| if (args.length < 2) { | ||
| System.err.println("Usage: DirectKafkaWordCount <broker list> <topic>") | ||
| System.exit(1) | ||
| } | ||
|
|
||
| StreamingExamples.setStreamingLogLevels() | ||
|
|
||
| val Array(brokerList, topics) = args | ||
| val sparkConf = new SparkConf().setAppName("DirectKafkaWordCount") | ||
| val ssc = new StreamingContext(sparkConf, Seconds(2)) | ||
| ssc.checkpoint("checkpoint") | ||
|
|
||
| val topicsSet = topics.split(",").toSet | ||
| val kafkaParams = Map[String, String]("metadata.broker.list" -> brokerList) | ||
| val lines = KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( | ||
| ssc, kafkaParams, topicsSet).map(_._2) | ||
| val words = lines.flatMap(_.split(" ")) | ||
| val wordCounts = words.map(x => (x, 1L)).reduceByKey(_ + _) | ||
| wordCounts.print() | ||
|
|
||
| ssc.start() | ||
| ssc.awaitTermination() | ||
| } | ||
| } | ||
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
201 changes: 201 additions & 0 deletions
201
external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,201 @@ | ||
| /* | ||
| * Licensed to the Apache Software Foundation (ASF) under one or more | ||
| * contributor license agreements. See the NOTICE file distributed with | ||
| * this work for additional information regarding copyright ownership. | ||
| * The ASF licenses this file to You under the Apache License, Version 2.0 | ||
| * (the "License"); you may not use this file except in compliance with | ||
| * the License. You may obtain a copy of the License at | ||
| * | ||
| * http://www.apache.org/licenses/LICENSE-2.0 | ||
| * | ||
| * Unless required by applicable law or agreed to in writing, software | ||
| * distributed under the License is distributed on an "AS IS" BASIS, | ||
| * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. | ||
| * See the License for the specific language governing permissions and | ||
| * limitations under the License. | ||
| */ | ||
|
|
||
| package org.apache.spark.streaming.kafka | ||
|
|
||
| import java.io.File | ||
|
|
||
| import scala.collection.mutable | ||
| import scala.concurrent.duration._ | ||
| import scala.language.postfixOps | ||
|
|
||
| import kafka.serializer.StringDecoder | ||
| import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} | ||
| import org.scalatest.concurrent.{Eventually, Timeouts} | ||
|
|
||
| import org.apache.spark.SparkConf | ||
| import org.apache.spark.rdd.RDD | ||
| import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} | ||
| import org.apache.spark.streaming.dstream.{DStream, InputDStream} | ||
| import org.apache.spark.util.Utils | ||
|
|
||
| class DirectKafkaStreamSuite extends KafkaStreamSuiteBase | ||
|
Contributor
Author
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. Just renamed the file and testsuite class from KafkaDirectStreamSuite to DirectKafkaStreamSuite, but Git/Github considered it to be a move. The first unit test is unmodified. |
||
| with BeforeAndAfter with BeforeAndAfterAll with Eventually { | ||
| val sparkConf = new SparkConf() | ||
| .setMaster("local[4]") | ||
| .setAppName(this.getClass.getSimpleName) | ||
|
|
||
| val brokerHost = "localhost" | ||
|
|
||
| val kafkaParams = Map( | ||
| "metadata.broker.list" -> s"$brokerHost:$brokerPort", | ||
| "auto.offset.reset" -> "smallest" | ||
| ) | ||
|
|
||
| var ssc: StreamingContext = _ | ||
| var testDir: File = _ | ||
|
|
||
| override def beforeAll { | ||
| setupKafka() | ||
| } | ||
|
|
||
| override def afterAll { | ||
| tearDownKafka() | ||
| } | ||
|
|
||
| after { | ||
| if (ssc != null) { | ||
| ssc.stop() | ||
| } | ||
| if (testDir != null) { | ||
| Utils.deleteRecursively(testDir) | ||
| } | ||
| } | ||
|
|
||
| test("basic receiving with multiple topics") { | ||
| val topics = Set("newA", "newB") | ||
| val data = Map("a" -> 7, "b" -> 9) | ||
| topics.foreach { t => | ||
| createTopic(t) | ||
| produceAndSendMessage(t, data) | ||
| } | ||
| ssc = new StreamingContext(sparkConf, Milliseconds(200)) | ||
| val stream = withClue("Error creating direct stream") { | ||
| KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( | ||
| ssc, kafkaParams, topics) | ||
| } | ||
| var total = 0L | ||
|
|
||
| stream.foreachRDD { rdd => | ||
| // Get the offset ranges in the RDD | ||
| val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges | ||
| val collected = rdd.mapPartitionsWithIndex { (i, iter) => | ||
| // For each partition, get size of the range in the partition, | ||
| // and the number of items in the partition | ||
| val off = offsets(i) | ||
| val all = iter.toSeq | ||
| val partSize = all.size | ||
| val rangeSize = off.untilOffset - off.fromOffset | ||
| Iterator((partSize, rangeSize)) | ||
| }.collect | ||
|
|
||
| // Verify whether number of elements in each partition | ||
| // matches with the corresponding offset range | ||
| collected.foreach { case (partSize, rangeSize) => | ||
| assert(partSize === rangeSize, "offset ranges are wrong") | ||
| } | ||
| total += collected.size // Add up all the collected items | ||
| } | ||
| ssc.start() | ||
| eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { | ||
| assert(total === data.values.sum * topics.size, "didn't get all messages") | ||
| } | ||
| ssc.stop() | ||
| } | ||
|
|
||
| // Test to verify the offset ranges can be recovered from the checkpoints | ||
| test("offset recovery") { | ||
| val topic = "recovery" | ||
| createTopic(topic) | ||
| testDir = Utils.createTempDir() | ||
|
|
||
| // Send data to Kafka and wait for it to be received | ||
| def sendDataAndWaitForReceive(data: Seq[Int]) { | ||
| val strings = data.map { _.toString} | ||
| produceAndSendMessage(topic, strings.map { _ -> 1}.toMap) | ||
| eventually(timeout(10 seconds), interval(50 milliseconds)) { | ||
| assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) | ||
| } | ||
| } | ||
|
|
||
| // Setup the streaming context | ||
| ssc = new StreamingContext(sparkConf, Milliseconds(100)) | ||
| val kafkaStream = withClue("Error creating direct stream") { | ||
| KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( | ||
| ssc, kafkaParams, Set(topic)) | ||
| } | ||
| val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } | ||
| val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => | ||
| Some(values.sum + state.getOrElse(0)) | ||
| } | ||
| ssc.checkpoint(testDir.getAbsolutePath) | ||
|
|
||
| // This is to collect the raw data received from Kafka | ||
| kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => | ||
| val data = rdd.map { _._2 }.collect() | ||
| DirectKafkaStreamSuite.collectedData.appendAll(data) | ||
| } | ||
|
|
||
| // This is ensure all the data is eventually receiving only once | ||
| stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => | ||
| rdd.collect().headOption.foreach { x => DirectKafkaStreamSuite.total = x._2 } | ||
| } | ||
| ssc.start() | ||
|
|
||
| // Send some data and wait for them to be received | ||
| for (i <- (1 to 10).grouped(4)) { | ||
| sendDataAndWaitForReceive(i) | ||
| } | ||
|
|
||
| // Verify that offset ranges were generated | ||
| val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) | ||
| assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") | ||
| assert( | ||
| offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, | ||
| "starting offset not zero" | ||
| ) | ||
| ssc.stop() | ||
| logInfo("====== RESTARTING ========") | ||
|
|
||
| // Recover context from checkpoints | ||
| ssc = new StreamingContext(testDir.getAbsolutePath) | ||
| val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] | ||
|
|
||
| // Verify offset ranges have been recovered | ||
| val recoveredOffsetRanges = getOffsetRanges(recoveredStream) | ||
| assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") | ||
| val earlierOffsetRangesAsSets = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } | ||
| assert( | ||
| recoveredOffsetRanges.forall { or => | ||
| earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) | ||
| }, | ||
| "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() | ||
| sendDataAndWaitForReceive(11 to 20) | ||
| eventually(timeout(10 seconds), interval(50 milliseconds)) { | ||
| assert(DirectKafkaStreamSuite.total === (1 to 20).sum) | ||
| } | ||
| ssc.stop() | ||
| } | ||
|
|
||
| /** Get the generated offset ranges from the DirectKafkaStream */ | ||
| private def getOffsetRanges[K, V]( | ||
| kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { | ||
| kafkaStream.generatedRDDs.mapValues { rdd => | ||
| rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges | ||
| }.toSeq.sortBy { _._1 } | ||
| } | ||
| } | ||
|
|
||
| object DirectKafkaStreamSuite { | ||
| val collectedData = new mutable.ArrayBuffer[String]() | ||
| var total = -1L | ||
| } | ||
92 changes: 0 additions & 92 deletions
92
external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaDirectStreamSuite.scala
This file was deleted.
Oops, something went wrong.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
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.
These are kafka servers, not zookeeper servers
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.
Right. Thanks!