From 048b5216c913ed7f70783aced3b9d37629c5d521 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Mon, 7 Dec 2015 12:59:51 +0200 Subject: [PATCH 01/16] [SPARK-12177] [STREAMING] Update KafkaDStreams to new Kafka 0.9 Consumer API --- examples/pom.xml | 5 + .../examples/streaming/KafkaWordCount.scala | 66 ++- .../streaming/v09DirectKafkaWordCount.scala | 89 +++ external/kafka-assembly/pom.xml | 5 + external/kafka-v09/pom.xml | 122 ++++ .../spark/streaming/kafka/v09/Broker.scala | 66 +++ .../kafka/v09/DirectKafkaInputDStream.scala | 198 +++++++ .../streaming/kafka/v09/KafkaCluster.scala | 206 +++++++ .../kafka/v09/KafkaInputDStream.scala | 130 +++++ .../spark/streaming/kafka/v09/KafkaRDD.scala | 208 +++++++ .../kafka/v09/KafkaRDDPartition.scala | 37 ++ .../streaming/kafka/v09/KafkaTestUtils.scala | 277 +++++++++ .../streaming/kafka/v09/KafkaUtils.scala | 538 ++++++++++++++++++ .../streaming/kafka/v09/OffsetRange.scala | 109 ++++ .../kafka/v09/ReliableKafkaReceiver.scala | 313 ++++++++++ .../kafka/v09/JavaDirectKafkaStreamSuite.java | 181 ++++++ .../kafka/v09/JavaKafkaRDDSuite.java | 156 +++++ .../kafka/v09/JavaKafkaStreamSuite.java | 141 +++++ .../src/test/resources/log4j.properties | 28 + .../kafka/v09/DirectKafkaStreamSuite.scala | 515 +++++++++++++++++ .../kafka/v09/KafkaClusterSuite.scala | 67 +++ .../streaming/kafka/v09/KafkaRDDSuite.scala | 184 ++++++ .../kafka/v09/KafkaStreamSuite.scala | 90 +++ .../kafka/v09/ReliableKafkaStreamSuite.scala | 158 +++++ pom.xml | 1 + 25 files changed, 3878 insertions(+), 12 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala create mode 100644 external/kafka-v09/pom.xml create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/Broker.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala create mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala create mode 100644 external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java create mode 100644 external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java create mode 100644 external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java create mode 100644 external/kafka-v09/src/test/resources/log4j.properties create mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala create mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala create mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala create mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala create mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala diff --git a/examples/pom.xml b/examples/pom.xml index f5ab2a7fdc09..6852ba93cc1d 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -102,6 +102,11 @@ spark-streaming-kafka_${scala.binary.version} ${project.version} + + org.apache.spark + spark-streaming-kafka-v09_${scala.binary.version} + ${project.version} + org.apache.hbase hbase-testing-util diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index b40d17e9c2fa..6fd331bd3228 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -20,6 +20,7 @@ package org.apache.spark.examples.streaming import java.util.HashMap +import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer, ProducerRecord} import org.apache.spark.streaming._ @@ -27,18 +28,18 @@ import org.apache.spark.streaming.kafka._ import org.apache.spark.SparkConf /** - * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: KafkaWordCount - * is a list of one or more zookeeper servers that make quorum - * is the name of kafka consumer group - * is a list of one or more kafka topics to consume from - * is the number of threads the kafka consumer should use - * - * Example: - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.KafkaWordCount zoo01,zoo02,zoo03 \ - * my-consumer-group topic1,topic2 1` - */ + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: KafkaWordCount + * is a list of one or more zookeeper servers that make quorum + * is the name of kafka consumer group + * is a list of one or more kafka topics to consume from + * is the number of threads the kafka consumer should use + * + * Example: + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.KafkaWordCount zoo01,zoo02,zoo03 \ + * my-consumer-group topic1,topic2 1` + */ object KafkaWordCount { def main(args: Array[String]) { if (args.length < 4) { @@ -65,6 +66,47 @@ object KafkaWordCount { } } +/** + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: NewKafkaWordCount + * is a list of one or more zookeeper servers that make quorum + * is the name of kafka consumer group + * is a list of one or more kafka topics to consume from + * is the number of threads the kafka consumer should use + * + * Example: + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.NewKafkaWordCount broker1,broker2,broker3 \ + * my-consumer-group topic1,topic2 1` + */ +object v09KafkaWordCount { + def main(args: Array[String]) { + import org.apache.spark.streaming.kafka.v09._ + + if (args.length < 4) { + System.err.println("Usage: v09KafkaWordCount ") + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + val Array(brokers, group, topics, numThreads) = args + val sparkConf = new SparkConf().setAppName("v09KafkaWordCount") + val ssc = new StreamingContext(sparkConf, Seconds(2)) + ssc.checkpoint("checkpoint") + + val topicMap = topics.split(",").map((_, numThreads.toInt)).toMap + val lines = KafkaUtils.createStream(ssc, brokers, group, topicMap).map(_._2) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1L)) + .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2) + wordCounts.print() + + ssc.start() + ssc.awaitTermination() + } +} + // Produces some random words between 1 and 100. object KafkaWordCountProducer { diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala new file mode 100644 index 000000000000..8d5485992718 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala @@ -0,0 +1,89 @@ +/* + * 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. + */ + +// scalastyle:off println +package org.apache.spark.examples.streaming + +import kafka.serializer.StringDecoder +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.serialization.StringDeserializer + +import org.apache.spark.streaming._ +import org.apache.spark.streaming.kafka.v09._ +import org.apache.spark.SparkConf + +/** + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: v09DirectKafkaWordCount + * is a list of one or more Kafka brokers + * is a list of one or more kafka topics to consume from + * + * Example: + * $ bin/run-example streaming.v09DirectKafkaWordCount broker1-host:port,broker2-host:port \ + * topic1,topic2 + */ +object v09DirectKafkaWordCount { + def main(args: Array[String]) { + if (args.length < 2) { + System.err.println(s""" + |Usage: v09DirectKafkaWordCount + | is a list of one or more Kafka brokers + | is a list of one or more kafka topics to consume from + | comsumer group id + | latest (default) or earliest + | time in seconds + | time in milliseconds + | + """.stripMargin) + System.exit(1) + } + + StreamingExamples.setStreamingLogLevels() + + val Array(brokers, topics, groupId, offsetReset, batchInterval, pollTimeout) = args + + // Create context with 2 second batch interval + val sparkConf = new SparkConf().setAppName("v09DirectKafkaWordCount") + val ssc = new StreamingContext(sparkConf, Seconds(batchInterval.toInt)) + val reset = if (offsetReset.isEmpty) "latest" else offsetReset + + // Create direct kafka stream with brokers and topics + val topicsSet = topics.split(",").toSet + val kafkaParams = Map[String, String]( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> brokers, + ConsumerConfig.GROUP_ID_CONFIG -> groupId, + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> reset, + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG -> "false", + "spark.kafka.poll.time" -> pollTimeout) + val messages = KafkaUtils.createDirectStream[String, String](ssc, kafkaParams, topicsSet) + + // Get the lines, split them into words, count the words and print + val lines = messages.map(_._2) + val words = lines.flatMap(_.split(" ")) + val wordCounts = words.map(x => (x, 1L)).reduceByKey(_ + _) + wordCounts.print() + + // Start the computation + ssc.start() + ssc.awaitTermination() + } +} +// scalastyle:on println diff --git a/external/kafka-assembly/pom.xml b/external/kafka-assembly/pom.xml index a9ed39ef8c9a..65f65b91182b 100644 --- a/external/kafka-assembly/pom.xml +++ b/external/kafka-assembly/pom.xml @@ -41,6 +41,11 @@ spark-streaming-kafka_${scala.binary.version} ${project.version} + + org.apache.spark + spark-streaming-kafka-v09_${scala.binary.version} + ${project.version} + org.apache.spark spark-streaming_${scala.binary.version} diff --git a/external/kafka-v09/pom.xml b/external/kafka-v09/pom.xml new file mode 100644 index 000000000000..a09b54466690 --- /dev/null +++ b/external/kafka-v09/pom.xml @@ -0,0 +1,122 @@ + + 4.0.0 + + + org.apache.spark + spark-parent_2.10 + 1.6.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-v09_2.10 + + streaming-kafka-v09 + + jar + Spark Project External Kafka v09 + http://spark.apache.org/ + + + + com.101tec + zkclient + 0.6 + + + com.yammer.metrics + metrics-core + 2.2.0 + + + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.kafka + kafka_${scala.binary.version} + 0.9.0.0 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + net.sf.jopt-simple + jopt-simple + + + org.slf4j + slf4j-simple + + + org.apache.zookeeper + zookeeper + + + + + org.apache.kafka + kafka-clients + 0.9.0.0 + + + com.sun.jmx + jmxri + + + com.sun.jdmk + jmxtools + + + net.sf.jopt-simple + jopt-simple + + + org.slf4j + slf4j-simple + + + org.apache.zookeeper + zookeeper + + + + + net.sf.jopt-simple + jopt-simple + 3.2 + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.apache.spark + spark-test-tags_${scala.binary.version} + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/Broker.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/Broker.scala new file mode 100644 index 000000000000..0826cd027e50 --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/Broker.scala @@ -0,0 +1,66 @@ +/* + * 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.v09 + +import org.apache.spark.annotation.Experimental + +/** + * Represents the host and port info for a Kafka broker. + * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID. + */ +final class Broker private( + /** Broker's hostname */ + val host: String, + /** Broker's port */ + val port: Int) extends Serializable { + override def equals(obj: Any): Boolean = obj match { + case that: Broker => + this.host == that.host && + this.port == that.port + case _ => false + } + + override def hashCode: Int = { + 41 * (41 + host.hashCode) + port + } + + override def toString(): String = { + s"Broker($host, $port)" + } +} + +/** + * :: Experimental :: + * Companion object that provides methods to create instances of [[Broker]]. + */ +@Experimental +object Broker { + def create(host: String, port: Int): Broker = + new Broker(host, port) + + def apply(host: String, port: Int): Broker = + new Broker(host, port) + + def unapply(broker: Broker): Option[(String, Int)] = { + if (broker == null) { + None + } else { + Some((broker.host, broker.port)) + } + } +} diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala new file mode 100644 index 000000000000..4416f38c235d --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala @@ -0,0 +1,198 @@ +/* + * 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.v09 + +import kafka.common.TopicAndPartition +import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.spark.Logging +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.scheduler.rate.RateEstimator +import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} +import org.apache.spark.streaming.{StreamingContext, Time} + +import scala.collection.mutable +import scala.reflect.ClassTag + +/** + * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * Starting offsets are specified in advance, + * and this DStream is not responsible for committing offsets, + * so that you can control exactly-once semantics. + * For an easy interface to Kafka-managed offsets, + * see {@link org.apache.spark.streaming.kafka.KafkaCluster} + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + */ +private[streaming] +class DirectKafkaInputDStream[ +K: ClassTag, +V: ClassTag, +R: ClassTag]( + @transient ssc_ : StreamingContext, + val kafkaParams: Map[String, String], + val fromOffsets: Map[TopicAndPartition, Long], + messageHandler: ConsumerRecord[K, V] => R + ) extends InputDStream[R](ssc_) with Logging { + + val maxRetries = context.sparkContext.getConf.getInt( + "spark.streaming.kafka.maxRetries", 1) + + // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") + private[streaming] override def name: String = s"Kafka 0.9 direct stream [$id]" + + protected[streaming] override val checkpointData = + new DirectKafkaInputDStreamCheckpointData + + + /** + * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. + */ + override protected[streaming] val rateController: Option[RateController] = { + if (RateController.isBackPressureEnabled(ssc.conf)) { + Some(new DirectKafkaRateController(id, + RateEstimator.create(ssc.conf, ssc_.graph.batchDuration))) + } else { + None + } + } + + protected var kafkaCluster = new KafkaCluster[K, V](kafkaParams) + + private val maxRateLimitPerPartition: Int = context.sparkContext.getConf.getInt( + "spark.streaming.kafka.maxRatePerPartition", 0) + protected def maxMessagesPerPartition: Option[Long] = { + val estimatedRateLimit = rateController.map(_.getLatestRate().toInt) + val numPartitions = currentOffsets.keys.size + + val effectiveRateLimitPerPartition = estimatedRateLimit + .filter(_ > 0) + .map { limit => + if (maxRateLimitPerPartition > 0) { + Math.min(maxRateLimitPerPartition, (limit / numPartitions)) + } else { + limit / numPartitions + } + }.getOrElse(maxRateLimitPerPartition) + + if (effectiveRateLimitPerPartition > 0) { + val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 + Some((secsPerBatch * effectiveRateLimitPerPartition).toLong) + } else { + None + } + } + + protected var currentOffsets = fromOffsets + + protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = { + val topicPartition = fromOffsets.map(fo => fo._1).toSet + kafkaCluster.getLatestOffsets(topicPartition).right.get + } + + // limits the maximum number of messages per partition + protected def clamp(leaderOffsets: Map[TopicAndPartition, Long] + ): Map[TopicAndPartition, Long] = { + maxMessagesPerPartition.map { mmp => + leaderOffsets.map { case (tp, lo) => + tp -> (Math.min(currentOffsets(tp) + mmp, lo)) + } + }.getOrElse(leaderOffsets) + } + + override def compute(validTime: Time): Option[KafkaRDD[K, V, R]] = { + val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) + val rdd = KafkaRDD[K, V, R]( + context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) + + // Report the record number and metadata of this batch interval to InputInfoTracker. + val offsetRanges = currentOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo) + } + val description = offsetRanges.filter { offsetRange => + // Don't display empty ranges. + offsetRange.fromOffset != offsetRange.untilOffset + }.map { offsetRange => + s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + + s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" + }.mkString("\n") + // Copy offsetRanges to immutable.List to prevent from being modified by the user + val metadata = Map( + "offsets" -> offsetRanges.toList, + StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) + val inputInfo = StreamInputInfo(id, rdd.count, metadata) + ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) + + currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2) + Some(rdd) + } + + override def start(): Unit = { + } + + def stop(): Unit = { + if (kafkaCluster != null) { + kafkaCluster.close() + kafkaCluster = null + } + } + + private[streaming] + class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { + def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { + data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] + } + + override def update(time: Time) { + batchForTime.clear() + generatedRDDs.foreach { kv => + val a = kv._2.asInstanceOf[KafkaRDD[K, V, R]].offsetRanges.map(_.toTuple).toArray + batchForTime += kv._1 -> a + } + } + + override def cleanup(time: Time) { } + + override def restore() { + // this is assuming that the topics don't change during execution, which is true currently + val topics = fromOffsets.keySet + + batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => + logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") + generatedRDDs += t -> new KafkaRDD[K, V, R]( + context.sparkContext, kafkaParams, b.map(OffsetRange(_)), messageHandler) + } + } + } + + /** + * A RateController to retrieve the rate from RateEstimator. + */ + private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + } +} diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala new file mode 100644 index 000000000000..9980d69ca597 --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala @@ -0,0 +1,206 @@ +/* + * 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.v09 + +import java.util +import java.util.{Properties, Collections} + +import kafka.common.TopicAndPartition +import org.apache.kafka.clients.consumer.{KafkaConsumer, OffsetAndMetadata} +import org.apache.kafka.common.TopicPartition +import org.apache.spark.SparkException +import org.apache.spark.streaming.kafka.v09.KafkaCluster.SeekType.SeekType + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.reflect._ +import scala.util.control.NonFatal + +/** + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form + */ +private[spark] +class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String]) + extends Serializable { + import KafkaCluster.{Err, SeekType, toTopicPart} + + def getLatestOffsets(topicPartitions: Set[TopicAndPartition]): + Either[Err, Map[TopicAndPartition, Long]] = { + seek(topicPartitions, SeekType.End) + } + + def getEarliestOffsets(topicPartitions: Set[TopicAndPartition]): + Either[Err, Map[TopicAndPartition, Long]] = { + seek(topicPartitions, SeekType.Beginning) + } + + def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = { + val errs = new Err + var result: Either[Err, Set[TopicAndPartition]] = null + + withConsumer(errs)(consumer => { + try { + val partInfo = topics.flatMap(topic => consumer.partitionsFor(topic).asScala) + val topicPartitions: Set[TopicAndPartition] = partInfo.map { partition => + new TopicAndPartition(partition.topic(), partition.partition()) + } + result = Right(topicPartitions) + } catch { + case NonFatal(e) => { + errs.append(e) + result = Left(errs) + } + } + }) + result + } + + def setConsumerOffsets(offsets: Map[TopicAndPartition, Long]): Unit = { + val errs = new Err + val topicPartOffsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() + val topicAndPartition = offsets.map(tpl => tpl._1).toSeq + + withConsumer(errs)(consumer => { + consumer.assign(Collections.emptyList[TopicPartition]) + consumer.assign(topicAndPartition.map(tp => toTopicPart(tp)).asJava) + + for ((topicAndPart, offset) <- offsets) { + val topicPartition = toTopicPart(topicAndPart) + val offsetAndMetadata = new OffsetAndMetadata(offset) + topicPartOffsets.put(topicPartition, offsetAndMetadata) + } + + consumer.commitSync(topicPartOffsets) + }) + } + + def getCommittedOffset(topicAndPartition: TopicAndPartition): Either[Err, Long] = { + val errs = new Err + var result: Either[Err, Long] = null + + withConsumer(errs)(consumer => { + try { + val topicPartition = toTopicPart(topicAndPartition) + consumer.assign(util.Arrays.asList(topicPartition)) + result = Right(consumer.committed(topicPartition).offset()) + } catch { + case NonFatal(e) => { + errs.append(e) + result = Left(errs) + } + } + }) + result + } + + def getCommittedOffsets(topicAndPartitions: Set[TopicAndPartition]): + Either[Err, Map[TopicAndPartition, Long]] = { + val errs = new Err + var result: Either[Err, Map[TopicAndPartition, Long]] = null + + withConsumer(errs)(consumer => { + try { + val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) + consumer.assign(topicPartitions.toList.asJava) + result = Right(topicAndPartitions.map( + tp => tp -> (consumer.committed(toTopicPart(tp))).offset()).toMap) + } catch { + case NonFatal(e) => { + errs.append(e) + result = Left(errs) + } + } + }) + result + } + + def seek(topicAndPartitions: Set[TopicAndPartition], seekType: SeekType): + Either[Err, Map[TopicAndPartition, Long]] = { + val errs = new Err + var result: Either[Err, Map[TopicAndPartition, Long]] = null + + withConsumer(errs)(consumer => { + try { + val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) + consumer.assign(topicPartitions.toList.asJava) + seekType match { + case SeekType.Beginning => consumer.seekToBeginning(topicPartitions.toArray: _*) + case SeekType.End => consumer.seekToEnd(topicPartitions.toArray: _*) + } + result = Right(topicAndPartitions.map( + tp => tp -> (consumer.position(toTopicPart(tp))) + ).toMap) + } catch { + case NonFatal(e) => { + errs.append(e) + result = Left(errs) + } + } + }) + result + + } + + private def withConsumer(errs: Err)(fn: KafkaConsumer[K, V] => Any): Unit = { + var consumer: KafkaConsumer[K, V] = null + try { + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + consumer = new KafkaConsumer[K, V](props) + fn(consumer) + } finally { + if (consumer != null) { + consumer.close() + } + } + } + + def close(): Unit = { + } + +} + +object KafkaCluster { + + object SeekType extends Enumeration { + type SeekType = Value + val Beginning, End = Value + } + + type Err = ArrayBuffer[Throwable] + + /** If the result is right, return it, otherwise throw SparkException */ + def checkErrors[T](result: Either[Err, T]): T = { + result.fold( + errs => throw new SparkException(errs.mkString("\n")), + ok => ok + ) + } + + def toTopicPart(topicAndPartition: TopicAndPartition): TopicPartition = { + new TopicPartition(topicAndPartition.topic, topicAndPartition.partition) + } + + def toTopicAndPart(topicPartition: TopicPartition): TopicAndPartition = { + TopicAndPartition(topicPartition.topic, topicPartition.partition) + } +} diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala new file mode 100644 index 000000000000..a7b251ef7231 --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala @@ -0,0 +1,130 @@ +/* + * 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.v09 + +import java.util.{Collections, Properties} + +import org.apache.kafka.clients.consumer.{ConsumerRecord, ConsumerRecords, KafkaConsumer} +import org.apache.spark.Logging +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.ThreadUtils + +import scala.collection.Map +import scala.reflect.ClassTag + +/** + * Input stream that pulls messages from a Kafka Broker. + * + * @param kafkaParams Map of kafka configuration parameters. + * See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + */ +private[streaming] +class KafkaInputDStream[ +K: ClassTag, +V: ClassTag]( + @transient ssc_ : StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + useReliableReceiver: Boolean, + storageLevel: StorageLevel + ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { + + def getReceiver(): Receiver[(K, V)] = { + if (!useReliableReceiver) { + logInfo("[!] Using 0.9 KafkaReceiver") + new KafkaReceiver[K, V](kafkaParams, topics, storageLevel) + } else { + logInfo("[!] Using 0.9 ReliableKafkaReceiver") + new ReliableKafkaReceiver[K, V](kafkaParams, topics, storageLevel) + } + } +} + +private[streaming] +class KafkaReceiver[ +K: ClassTag, +V: ClassTag]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ) extends Receiver[(K, V)](storageLevel) with Logging { + + private var kafkaCluster: KafkaCluster[_, _] = null + + private val KAFKA_DEFAULT_POLL_TIME: String = "0" + private val pollTime = kafkaParams.get("spark.kafka.poll.time") + .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt + + def onStop() { + } + + def onStart() { + + logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) + + // Kafka connection properties + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + kafkaCluster = new KafkaCluster[K, V](kafkaParams.toMap) + + val executorPool = ThreadUtils.newDaemonFixedThreadPool(topics.values.sum, + "KafkaMessageHandler") + try { + // Start the messages handler for each partition + val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet).right.toOption + val iter = topicAndPartitions.get.iterator + + while (iter.hasNext) { + val topicAndPartition = iter.next() + val newConsumer = new KafkaConsumer[K, V](props) + newConsumer.subscribe(Collections.singletonList[String](topicAndPartition.topic)) + executorPool.submit(new MessageHandler(newConsumer)) + } + } finally { + executorPool.shutdown() // Just causes threads to terminate after work is done + } + } + + // Handles Kafka messages + private class MessageHandler(consumer: KafkaConsumer[K, V]) + extends Runnable { + def run() { + logInfo("Starting MessageHandler.") + try { + while (true) { + val records: ConsumerRecords[K, V] = this.consumer.poll(pollTime) + val iterator = records.iterator() + while (iterator.hasNext) { + val record: ConsumerRecord[K, V] = iterator.next() + store((record.key, record.value())) + } + } + } catch { + case e: Throwable => reportError("Error handling message; exiting", e) + } + } + } + +} + diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala new file mode 100644 index 000000000000..00042409f918 --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala @@ -0,0 +1,208 @@ +/* + * 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.v09 + +import java.util.{Collections, Properties} + +import kafka.common.TopicAndPartition +import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} +import org.apache.kafka.common.TopicPartition +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.kafka.v09.KafkaCluster.toTopicPart +import org.apache.spark.util.NextIterator +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.ClassTag + +/** + * A batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + */ +private[kafka] +class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark]( + sc: SparkContext, + kafkaParams: Map[String, String], + val offsetRanges: Array[OffsetRange], + messageHandler: ConsumerRecord[K, V] => R + ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { + + private val KAFKA_DEFAULT_POLL_TIME: String = "0" + private val pollTime = kafkaParams.get("spark.kafka.poll.time") + .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt + + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) + }.toArray + } + + override def count(): Long = offsetRanges.map(_.count).sum + + override def countApprox( + timeout: Long, + confidence: Double = 0.95 + ): PartialResult[BoundedDouble] = { + val c = count + new PartialResult(new BoundedDouble(c, 1.0, c, c), true) + } + + override def isEmpty(): Boolean = count == 0L + + override def take(num: Int): Array[R] = { + val nonEmptyPartitions = this.partitions + .map(_.asInstanceOf[KafkaRDDPartition]) + .filter(_.count > 0) + + if (num < 1 || nonEmptyPartitions.size < 1) { + return new Array[R](0) + } + + // Determine in advance how many messages need to be taken from each partition + val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => + val remain = num - result.values.sum + if (remain > 0) { + val taken = Math.min(remain, part.count) + result + (part.index -> taken.toInt) + } else { + result + } + } + + val buf = new ArrayBuffer[R] + val res = context.runJob( + this, + (tc: TaskContext, it: Iterator[R]) => it.take(parts(tc.partitionId)).toArray, + parts.keys.toArray) + res.foreach(buf ++= _) + buf.toArray + } + + private def errBeginAfterEnd(part: KafkaRDDPartition): String = + s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition}. " + + "You either provided an invalid fromOffset, or the Kafka topic has been damaged" + + private def errRanOutBeforeEnd(part: KafkaRDDPartition): String = + s"Ran out of messages before reaching ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates that messages may have been lost" + + private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String = + s"Got ${itemOffset} > ending offset ${part.untilOffset} " + + s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + + " This should not happen, and indicates a message may have been skipped" + + override def compute(thePart: Partition, context: TaskContext): Iterator[R] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) + if (part.fromOffset == part.untilOffset) { + log.info(s"Beginning offset ${part.fromOffset} is the same as ending offset " + + s"skipping ${part.topic} ${part.partition}") + Iterator.empty + } else { + new KafkaRDDIterator(part, context) + } + } + + private class KafkaRDDIterator( + part: KafkaRDDPartition, + context: TaskContext) extends NextIterator[R] { + + context.addTaskCompletionListener{ context => closeIfNeeded() } + + log.info(s"Computing topic ${part.topic}, partition ${part.partition} " + + s"offsets ${part.fromOffset} -> ${part.untilOffset}") + + val props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + + val consumer = new KafkaConsumer[K, V](props) + val tp = new TopicAndPartition(part.topic, part.partition) + consumer.assign(Collections.singletonList[TopicPartition](toTopicPart(tp))) + + var requestOffset = part.fromOffset + var iter: java.util.Iterator[ConsumerRecord[K, V]] = null + consumer.seek(toTopicPart(tp), requestOffset) + + override def close(): Unit = { + if (consumer != null) { + consumer.close() + } + } + + override def getNext(): R = { + if (iter == null || !iter.hasNext) { + iter = consumer.poll(pollTime).iterator() + } + + if (!iter.hasNext) { + finished = true + null.asInstanceOf[R] + } else { + val item: ConsumerRecord[K, V] = iter.next() + if (item.offset >= part.untilOffset) { + assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) + finished = true + null.asInstanceOf[R] + } else { + requestOffset = item.offset() + 1 + messageHandler(item) + } + } + } + } +} + +private[kafka] +object KafkaRDD { + + /** + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the batch + * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) + * ending point of the batch + */ + def apply[ + K: ClassTag, + V: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + untilOffsets: Map[TopicAndPartition, Long], + messageHandler: ConsumerRecord[K, V] => R + ): KafkaRDD[K, V, R] = { + val offsetRanges = fromOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo) + }.toArray + + new KafkaRDD[K, V, R](sc, kafkaParams, offsetRanges, messageHandler) + } +} diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala new file mode 100644 index 000000000000..df1e4be5c18f --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala @@ -0,0 +1,37 @@ +/* + * 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.v09 + +import org.apache.spark.Partition + +/** @param topic kafka topic name + * @param partition kafka partition id + * @param fromOffset inclusive starting offset + * @param untilOffset exclusive ending offset + */ +private[kafka] +class KafkaRDDPartition( + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long + ) extends Partition { + /** Number of messages this partition refers to */ + def count(): Long = untilOffset - fromOffset +} diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala new file mode 100644 index 000000000000..4f30d0d42c19 --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala @@ -0,0 +1,277 @@ +/* + * 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.v09 + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.concurrent.TimeoutException +import java.util.{Map => JMap, Properties} + +import kafka.admin.AdminUtils +import kafka.api.Request +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.ZkUtils +import org.I0Itec.zkclient.ZkClient +import org.apache.kafka.common.security.JaasUtils +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils +import org.apache.spark.{Logging, SparkConf} +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.language.postfixOps +import scala.util.control.NonFatal + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +private[kafka] class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 60000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkClient: ZkClient = _ + private var zkUtils: ZkUtils = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 9092 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkClient = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkClient).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkClient = ZkUtils.createZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout) + zkUtils = ZkUtils(zkClient, JaasUtils.isZkSecurityEnabled()) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration) + server = new KafkaServer(brokerConf) + server.startup() + (server, port) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server = null + } + + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (zkUtils != null) { + zkUtils.close() + zkUtils = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + AdminUtils.createTopic(zkUtils, topic, 1, 1) + // wait until metadata is propagated + waitUntilMetadataIsPropagated(topic, 0) + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Unit = { + producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) + producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) + producer.close() + producer = null + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerAddress) + props.put("serializer.class", classOf[StringEncoder].getName) + // wait for all in-sync replicas to ack sends + props.put("request.required.acks", "-1") + props + } + + // A simplified version of scalatest eventually, rewritten here to avoid adding extra test + // dependency + def eventually[T](timeout: Time, interval: Time)(func: => T): T = { + def makeAttempt(): Either[Throwable, T] = { + try { + Right(func) + } catch { + case e if NonFatal(e) => Left(e) + } + } + + val startTime = System.currentTimeMillis() + @tailrec + def tryAgain(attempt: Int): T = { + makeAttempt() match { + case Right(result) => result + case Left(e) => + val duration = System.currentTimeMillis() - startTime + if (duration < timeout.milliseconds) { + Thread.sleep(interval.milliseconds) + } else { + throw new TimeoutException(e.getMessage) + } + + tryAgain(attempt + 1) + } + } + + tryAgain(1) + } + + 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(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") + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } +} + diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala new file mode 100644 index 000000000000..2a228659f80d --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala @@ -0,0 +1,538 @@ +/* + * 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.v09 + +import java.io.OutputStream +import java.lang.{Integer => JInt, Long => JLong} +import java.util.{List => JList, Map => JMap, Set => JSet} + +import scala.reflect.ClassTag + +import com.google.common.base.Charsets.UTF_8 +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder} +import net.razorvine.pickle.{Opcodes, Pickler, IObjectPickler} +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.streaming.util.WriteAheadLogUtils +import org.apache.spark.{SparkContext, SparkException} + +import scala.collection.JavaConverters._ +import scala.reflect._ +import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} +import org.apache.spark.api.python.SerDeUtil +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java._ +import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} + +object KafkaUtils { + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param ssc StreamingContext object + * @param servers Broker servers (for Kafka 0.9) (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + */ + def createStream( + ssc: StreamingContext, + servers: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[(String, String)] = { + + val kafkaParams = Map[String, String]( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> servers, + ConsumerConfig.GROUP_ID_CONFIG -> groupId, + ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG -> "5000", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "1000" + ) + createStream[String, String]( + ssc, kafkaParams, topics, storageLevel) + + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param ssc StreamingContext object + * @param kafkaParams Map of kafka configuration parameters + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel Storage level to use for storing the received objects + */ + def createStream[K: ClassTag, V: ClassTag]( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel + ): ReceiverInputDStream[(K, V)] = { + val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) + new KafkaInputDStream[K, V](ssc, kafkaParams, topics, walEnabled, storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt] + ): JavaPairReceiverInputDStream[String, String] = { + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*)) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param jssc JavaStreamingContext object + * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). + * @param groupId The group id for this consumer. + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + */ + def createStream( + jssc: JavaStreamingContext, + zkQuorum: String, + groupId: String, + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[String, String] = { + createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), + storageLevel) + } + + /** + * Create an input stream that pulls messages from Kafka Brokers. + * @param jssc JavaStreamingContext object + * @param keyTypeClass Key type of DStream + * @param valueTypeClass value type of Dstream + * @param kafkaParams Map of kafka configuration parameters + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel RDD storage level. + */ + def createStream[K, V]( + jssc: JavaStreamingContext, + keyTypeClass: Class[K], + valueTypeClass: Class[V], + kafkaParams: JMap[String, String], + topics: JMap[String, JInt], + storageLevel: StorageLevel + ): JavaPairReceiverInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) + + createStream[K, V]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), + storageLevel) + } + + /** Make sure offsets are available in kafka, or throw an exception */ + private def checkOffsets( + kc: KafkaCluster[_, _], + offsetRanges: Array[OffsetRange]): Unit = { + val topics = offsetRanges.map(_.topicAndPartition).toSet + val result = for { + low <- kc.getEarliestOffsets(topics).right + high <- kc.getLatestOffsets(topics).right + } yield { + offsetRanges.filterNot { o => + low(o.topicAndPartition) <= o.fromOffset && + o.untilOffset <= high(o.topicAndPartition) + } + } + val badRanges = KafkaCluster.checkErrors(result) + if (!badRanges.isEmpty) { + throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) + } + } + + + def createRDD[ + K: ClassTag, + V: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange] + ): RDD[(K, V)] = sc.withScope { + val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) + val kc = new KafkaCluster[K, V](kafkaParams) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, (K, V)](sc, kafkaParams, offsetRanges, messageHandler) + } + + /** + * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param sc SparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param messageHandler Function for translating each message and metadata into the desired type + */ + def createRDD[ + K: ClassTag, + V: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange], + messageHandler: ConsumerRecord[K, V] => R + ): RDD[R] = sc.withScope { + val kc = new KafkaCluster[K, V](kafkaParams) + val cleanedHandler = sc.clean(messageHandler) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, R](sc, kafkaParams, offsetRanges, cleanedHandler) + } + + /** + * Create a RDD from Kafka using offset ranges for each topic and partition. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * specified in host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange] + ): JavaPairRDD[K, V] = jsc.sc.withScope { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + new JavaPairRDD(createRDD[K, V]( + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges)) + } + + /** + * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * specified in host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param messageHandler Function for translating each message and metadata into the desired type + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + recordClass: Class[R], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange], + messageHandler: JFunction[ConsumerRecord[K, V], R] + ): JavaRDD[R] = jsc.sc.withScope { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + createRDD[K, V, R]( + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, messageHandler.call _) + } + + +/** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag, + R: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + messageHandler: ConsumerRecord[K, V] => R + ): InputDStream[R] = { + val cleanedHandler = ssc.sc.clean(messageHandler) + new DirectKafkaInputDStream[K, V, R]( + ssc, kafkaParams, fromOffsets, messageHandler) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * to determine where the stream starts (defaults to "largest") + * @param topics Names of the topics to consume + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Set[String] + ): InputDStream[(K, V)] = { + val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) + val kc = new KafkaCluster[K, V](kafkaParams) + val reset = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).map(_.toLowerCase) + + val fromOffsets = if (reset == Some("earliest")) { + kc.getEarliestOffsets(kc.getPartitions(topics).right.get).right.get + } else { + kc.getLatestOffsets(kc.getPartitions(topics).right.get).right.get + } + + kc.close() + + new DirectKafkaInputDStream[K, V, (K, V)]( + ssc, kafkaParams, fromOffsets, messageHandler) + } + +/** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param recordClass Class of the records in DStream + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * specified in host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + */ + def createDirectStream[K, V, R]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + recordClass: Class[R], + kafkaParams: JMap[String, String], + fromOffsets: JMap[TopicAndPartition, JLong], + messageHandler: JFunction[ConsumerRecord[K, V], R] + ): JavaInputDStream[R] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) + createDirectStream[K, V, R]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Map(fromOffsets.asScala.mapValues { _.longValue() }.toSeq: _*), + cleanedHandler + ) + } + + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. For interoperability with Kafka monitoring tools that depend on + * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param kafkaParams Kafka + * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" + * to determine where the stream starts (defaults to "largest") + * @param topics Names of the topics to consume + */ + def createDirectStream[K, V]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + kafkaParams: JMap[String, String], + topics: JSet[String] + ): JavaPairInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + createDirectStream[K, V]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Set(topics.asScala.toSeq: _*) + ) + } + + def createOffsetRange(topic: String, partition: JInt, fromOffset: JLong, untilOffset: JLong + ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) + + def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = + TopicAndPartition(topic, partition) +} + +private object KafkaUtilsPythonHelper { + private var initialized = false + + def initialize(): Unit = { + SerDeUtil.initialize() + synchronized { + if (!initialized) { + new PythonMessageAndMetadataPickler().register() + initialized = true + } + } + } + + initialize() + + def picklerIterator(iter: Iterator[Any]): Iterator[Array[Byte]] = { + new SerDeUtil.AutoBatchedPickler(iter) + } + + case class PythonMessageAndMetadata( + topic: String, + partition: JInt, + offset: JLong, + key: Array[Byte], + message: Array[Byte]) + + class PythonMessageAndMetadataPickler extends IObjectPickler { + private val module = "pyspark.streaming.kafka" + + def register(): Unit = { + Pickler.registerCustomPickler(classOf[PythonMessageAndMetadata], this) + Pickler.registerCustomPickler(this.getClass, this) + } + + def pickle(obj: Object, out: OutputStream, pickler: Pickler) { + if (obj == this) { + out.write(Opcodes.GLOBAL) + out.write(s"$module\nKafkaMessageAndMetadata\n".getBytes(UTF_8)) + } else { + pickler.save(this) + val msgAndMetaData = obj.asInstanceOf[PythonMessageAndMetadata] + out.write(Opcodes.MARK) + pickler.save(msgAndMetaData.topic) + pickler.save(msgAndMetaData.partition) + pickler.save(msgAndMetaData.offset) + pickler.save(msgAndMetaData.key) + pickler.save(msgAndMetaData.message) + out.write(Opcodes.TUPLE) + out.write(Opcodes.REDUCE) + } + } + } +} diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala new file mode 100644 index 000000000000..ff245d3df4b6 --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala @@ -0,0 +1,109 @@ +/* + * 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.v09 + +import kafka.common.TopicAndPartition + +/** + * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the + * offset ranges in RDDs generated by the direct Kafka DStream (see + * [[KafkaUtils.createDirectStream()]]). + * {{{ + * KafkaUtils.createDirectStream(...).foreachRDD { rdd => + * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + * ... + * } + * }}} + */ +trait HasOffsetRanges { + def offsetRanges: Array[OffsetRange] +} + +/** + * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class + * can be created with `OffsetRange.create()`. + * @param topic Kafka topic name + * @param partition Kafka partition id + * @param fromOffset Inclusive starting offset + * @param untilOffset Exclusive ending offset + */ +final class OffsetRange private( + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long) extends Serializable { + import OffsetRange.OffsetRangeTuple + + /** Kafka TopicAndPartition object, for convenience */ + def topicAndPartition(): TopicAndPartition = TopicAndPartition(topic, partition) + + /** Number of messages this OffsetRange refers to */ + def count(): Long = untilOffset - fromOffset + + override def equals(obj: Any): Boolean = obj match { + case that: OffsetRange => + this.topic == that.topic && + this.partition == that.partition && + this.fromOffset == that.fromOffset && + this.untilOffset == that.untilOffset + case _ => false + } + + override def hashCode(): Int = { + toTuple.hashCode() + } + + override def toString(): String = { + s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" + } + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[streaming] + def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) +} + +/** + * Companion object the provides methods to create instances of [[OffsetRange]]. + */ +object OffsetRange { + def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def create( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset) + + def apply( + topicAndPartition: TopicAndPartition, + fromOffset: Long, + untilOffset: Long): OffsetRange = + new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + + /** this is to avoid ClassNotFoundException during checkpoint restore */ + private[kafka] + type OffsetRangeTuple = (String, Int, Long, Long) + + private[kafka] + def apply(t: OffsetRangeTuple) = + new OffsetRange(t._1, t._2, t._3, t._4) +} diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala new file mode 100644 index 000000000000..3946770dac91 --- /dev/null +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala @@ -0,0 +1,313 @@ +/* + * 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.v09 + +import java.util +import java.util.concurrent.locks.ReentrantLock +import java.util.concurrent.{ConcurrentHashMap, ThreadPoolExecutor} +import java.util.{Collections, Properties} + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} +import org.apache.spark.util.ThreadUtils +import org.apache.spark.{Logging, SparkEnv} + +import scala.collection.{Map, mutable} +import scala.reflect.ClassTag + +/** + * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager + * without loss. + * It is turned off by default and will be enabled when + * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver + * is that this receiver manages topic-partition/offset itself and updates the offset information + * after data is reliably stored as write-ahead log. Offsets will only be updated when data is + * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated. + * + * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset + * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams + * will not take effect. + */ +private[streaming] +class ReliableKafkaReceiver[ +K: ClassTag, +V: ClassTag]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel) + extends Receiver[(K, V)](storageLevel) with Logging { + + private val groupId = kafkaParams("group.id") + private val AUTO_OFFSET_COMMIT = ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + + private def conf = SparkEnv.get.conf + + private var kafkaCluster: KafkaCluster[_, _] = null + private val lock: ReentrantLock = new ReentrantLock() + + private var props: Properties = null + + private val KAFKA_DEFAULT_POLL_TIME: String = "0" + private val pollTime = kafkaParams.get("spark.kafka.poll.time") + .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt + + /** + * A HashMap to manage the offset for each topic/partition, this HashMap is called in + * synchronized block, so mutable HashMap will not meet concurrency issue. + */ + private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null + + /** A concurrent HashMap to store the stream block id and related offset snapshot. */ + private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null + + /** + * Manage the BlockGenerator in receiver itself for better managing block store and offset + * commit. + */ + private var blockGenerator: BlockGenerator = null + + /** Thread pool running the handlers for receiving message from multiple topics and partitions. */ + private var messageHandlerThreadPool: ThreadPoolExecutor = null + + private var topicAndPartitionConsumerMap: + mutable.HashMap[TopicAndPartition, KafkaConsumer[K, V]] = null + + private var consumerAndLockMap: + mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] = null + + override def onStart(): Unit = { + logInfo(s"Starting Kafka Consumer Stream with group: $groupId") + logWarning("[!] -> Starting 0.9 ReliableKafkaReceiver") + // Initialize the topic-partition / offset hash map. + topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] + + topicAndPartitionConsumerMap = new mutable.HashMap[TopicAndPartition, KafkaConsumer[K, V]] + consumerAndLockMap = new mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] + + // Initialize the stream block id / offset snapshot hash map. + blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]]() + + // Initialize the block generator for storing Kafka message. + blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) + + if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { + logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + + "otherwise we will manually set it to false to turn off auto offset commit in Kafka") + } + + props = new Properties() + kafkaParams.foreach(param => props.put(param._1, param._2)) + // Manually set "auto.commit.enable" to "false" no matter user explicitly set it to true, + // we have to make sure this property is set to false to turn off auto commit mechanism in + // Kafka. + props.setProperty(AUTO_OFFSET_COMMIT, "false") + + messageHandlerThreadPool = ThreadUtils.newDaemonFixedThreadPool( + topics.values.sum, "KafkaMessageHandler") + + blockGenerator.start() + + kafkaCluster = new KafkaCluster[K, V](kafkaParams.toMap) + + try { + // Start the messages handler for each partition + val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet).right.toOption + val iter = topicAndPartitions.get.iterator + while (iter.hasNext) { + val topicAndPartition = iter.next() + val newConsumer = new KafkaConsumer[K, V](props) + topicAndPartitionConsumerMap.put( + topicAndPartition, + newConsumer + ) + consumerAndLockMap.put(newConsumer, new ReentrantLock()) + newConsumer.subscribe(Collections.singletonList[String](topicAndPartition.topic)) + messageHandlerThreadPool.submit(new MessageHandler(newConsumer)) + } + } finally { + messageHandlerThreadPool.shutdown() // Just causes threads to terminate after work is done + } + } + + override def onStop(): Unit = { + if (messageHandlerThreadPool != null) { + messageHandlerThreadPool.shutdown() + messageHandlerThreadPool = null + } + + if (blockGenerator != null) { + blockGenerator.stop() + blockGenerator = null + } + + if (topicPartitionOffsetMap != null) { + topicPartitionOffsetMap.clear() + topicPartitionOffsetMap = null + } + + if (blockOffsetMap != null) { + blockOffsetMap.clear() + blockOffsetMap = null + } + } + + /** Store a Kafka message and the associated metadata as a tuple. */ + private def storeMessageAndMetadata( + msgAndMetadata: MessageAndMetadata[K, V]): Unit = { + val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) + val data = (msgAndMetadata.key, msgAndMetadata.message) + val metadata = (topicAndPartition, msgAndMetadata.offset) + blockGenerator.addDataWithCallback(data, metadata) + } + + private def storeConsumerRecord(consumerRecord: ConsumerRecord[K, V]): Unit = { + val topicAndPartition = TopicAndPartition(consumerRecord.topic, consumerRecord.partition) + val data = (consumerRecord.key, consumerRecord.value()) + val metadata = (topicAndPartition, consumerRecord.offset) + blockGenerator.addDataWithCallback(data, metadata) + } + + /** Update stored offset */ + private def updateOffset(topicAndPartition: TopicAndPartition, offset: Long): Unit = { + topicPartitionOffsetMap.put(topicAndPartition, offset) + } + + /** + * Remember the current offsets for each topic and partition. This is called when a block is + * generated. + */ + private def rememberBlockOffsets(blockId: StreamBlockId): Unit = { + // Get a snapshot of current offset map and store with related block id. + val offsetSnapshot = topicPartitionOffsetMap.toMap + blockOffsetMap.put(blockId, offsetSnapshot) + topicPartitionOffsetMap.clear() + } + + /** + * Store the ready-to-be-stored block and commit the related offsets to zookeeper. This method + * will try a fixed number of times to push the block. If the push fails, + * the receiver is stopped. + */ + private def storeBlockAndCommitOffset(blockId: StreamBlockId, + arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + var count = 0 + var pushed = false + var exception: Exception = null + while (!pushed && count <= 3) { + try { + store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) + pushed = true + } catch { + case ex: Exception => + count += 1 + exception = ex + } + } + if (pushed) { + Option(blockOffsetMap.get(blockId)).foreach(commitOffset) + blockOffsetMap.remove(blockId) + } else { + stop("Error while storing block into Spark", exception) + } + } + + /** + * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's + * metadata schema in Zookeeper. + */ + private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = { + val offsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() + for ((topicAndPart, offset) <- offsetMap) { + val kafkaConsumer = topicAndPartitionConsumerMap.get(topicAndPart).get // todo: remove get + + val topicPartition = new TopicPartition(topicAndPart.topic, topicAndPart.partition) + val offsetAndMetadata = new OffsetAndMetadata(offset) + offsets.put(topicPartition, offsetAndMetadata) + val lock = consumerAndLockMap(kafkaConsumer) + kafkaConsumer.wakeup() + lock.lock() + try { + kafkaConsumer.commitSync(offsets) + } finally { + lock.unlock() + } + } + + } + + /** Class to handle received Kafka message. */ + private final class MessageHandler(consumer: KafkaConsumer[K, V]) extends Runnable { + override def run(): Unit = { + var records: ConsumerRecords[K, V] = null + val lock = consumerAndLockMap(consumer) + while (!isStopped) { + try { + while (true) { + lock.lock() + try { + records = consumer.poll(pollTime) + } finally { + lock.unlock() + } + val iterator = records.iterator() + while (iterator.hasNext) { + val record: ConsumerRecord[K, V] = iterator.next() + storeConsumerRecord(record) + } + } + } catch { + case e: Exception => { + reportError("Error handling message", e) + } + + } + } + } + } + + /** Class to handle blocks generated by the block generator. */ + private final class GeneratedBlockHandler extends BlockGeneratorListener { + + def onAddData(data: Any, metadata: Any): Unit = { + // Update the offset of the data that was added to the generator + if (metadata != null) { + val (topicAndPartition, offset) = metadata.asInstanceOf[(TopicAndPartition, Long)] + updateOffset(topicAndPartition, offset) + } + } + + def onGenerateBlock(blockId: StreamBlockId): Unit = { + // Remember the offsets of topics/partitions when a block has been generated + rememberBlockOffsets(blockId) + } + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + // Store block and commit the blocks offset + storeBlockAndCommitOffset(blockId, arrayBuffer) + } + + def onError(message: String, throwable: Throwable): Unit = { + reportError(message, throwable) + } + } + +} diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java new file mode 100644 index 000000000000..17b5b1a8607a --- /dev/null +++ b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java @@ -0,0 +1,181 @@ +/* + * 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.v09; + + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; +import org.apache.spark.streaming.kafka.v09.HasOffsetRanges; +import org.apache.spark.streaming.kafka.v09.KafkaTestUtils; +import org.apache.spark.streaming.kafka.v09.KafkaUtils; +import org.apache.spark.streaming.kafka.v09.OffsetRange; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import scala.Tuple2; + +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +public class JavaDirectKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + } + + @After + public void tearDown() { + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaStream() throws InterruptedException { + final String topic1 = "new_topic1_testKafkaDirectStream"; + final String topic2 = "new_topic2_testKafkaDirectStream"; + // hold a reference to the current offset ranges, so it can be used downstream + final AtomicReference offsetRanges = new AtomicReference<>(); + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + HashSet sent = new HashSet(); + sent.addAll(Arrays.asList(topic1data)); + sent.addAll(Arrays.asList(topic2data)); + + HashMap kafkaParams = new HashMap(); + kafkaParams.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaTestUtils.brokerAddress()); + kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + kafkaParams.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + kafkaParams.put("spark.kafka.poll.time", "1000"); + + JavaDStream stream1 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + kafkaParams, + topicToSet(topic1) + ).transformToPair( + // Make sure you can get offset ranges from the rdd + new Function, JavaPairRDD>() { + @Override + public JavaPairRDD call(JavaPairRDD rdd) throws Exception { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(offsets[0].topic(), topic1); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) throws Exception { + return kv._2(); + } + } + ); + + JavaDStream stream2 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + String.class, + kafkaParams, + topicOffsetToMap(topic2, (long) 0), + new Function, String>() { + @Override + public String call(ConsumerRecord consumerRecord) throws Exception { + return consumerRecord.value(); + } + } + ); + JavaDStream unifiedStream = stream1.union(stream2); + + final Set result = Collections.synchronizedSet(new HashSet()); + unifiedStream.foreachRDD( + new Function, Void>() { + @Override + public Void call(JavaRDD rdd) throws Exception { + result.addAll(rdd.collect()); + for (OffsetRange o : offsetRanges.get()) { + System.out.println( + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() + ); + } + return null; + } + } + ); + ssc.start(); + long startTime = System.currentTimeMillis(); + boolean matches = false; + while (!matches && System.currentTimeMillis() - startTime < 20000) { + matches = sent.size() == result.size(); + Thread.sleep(50); + } + Assert.assertEquals(sent, result); + ssc.stop(); + } + + private HashSet topicToSet(String topic) { + HashSet topicSet = new HashSet(); + topicSet.add(topic); + return topicSet; + } + + private HashMap topicOffsetToMap(String topic, Long offsetToStart) { + HashMap topicMap = new HashMap(); + topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); + return topicMap; + } + + private String[] createTopicAndSendData(String topic) { + String[] data = {topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java new file mode 100644 index 000000000000..1d79fcea3738 --- /dev/null +++ b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java @@ -0,0 +1,156 @@ +/* + * 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.v09; + +import java.io.Serializable; +import java.util.HashMap; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.spark.streaming.kafka.v09.Broker; +import org.apache.spark.streaming.kafka.v09.KafkaTestUtils; +import org.apache.spark.streaming.kafka.v09.KafkaUtils; +import org.apache.spark.streaming.kafka.v09.OffsetRange; +import scala.Tuple2; + +import kafka.common.TopicAndPartition; +import kafka.message.MessageAndMetadata; +import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; + +public class JavaKafkaRDDSuite implements Serializable { + private transient JavaSparkContext sc = null; + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + sc = new JavaSparkContext(sparkConf); + } + + @After + public void tearDown() { + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaRDD() throws InterruptedException { + String topic1 = "new_topic1_testKafkaRDD"; + String topic2 = "new_topic2_testKafkaRDD"; + + String[] topic1data = createTopicAndSendData(topic1); + String[] topic2data = createTopicAndSendData(topic2); + + HashMap kafkaParams = new HashMap(); + kafkaParams.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaTestUtils.brokerAddress()); + kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + kafkaParams.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + kafkaParams.put("spark.kafka.poll.time", "1000"); + + OffsetRange[] offsetRanges = { + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) + }; + + HashMap emptyLeaders = new HashMap(); + HashMap leaders = new HashMap(); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); + Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); + leaders.put(new TopicAndPartition(topic1, 0), broker); + leaders.put(new TopicAndPartition(topic2, 0), broker); + + JavaRDD rdd1 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + kafkaParams, + offsetRanges + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) throws Exception { + return kv._2(); + } + } + ); + + JavaRDD rdd2 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + String.class, + kafkaParams, + offsetRanges, + new Function, String>() { + @Override + public String call(ConsumerRecord consumerRecord) throws Exception { + return consumerRecord.value(); + } + } + ); + + JavaRDD rdd3 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + String.class, + kafkaParams, + offsetRanges, + new Function, String>() { + @Override + public String call(ConsumerRecord consumerRecord) throws Exception { + return consumerRecord.value(); + } + } + ); + + // just making sure the java user apis work; the scala tests handle logic corner cases + long count1 = rdd1.count(); + long count2 = rdd2.count(); + long count3 = rdd3.count(); + Assert.assertTrue(count1 > 0); + Assert.assertEquals(count1, count2); + Assert.assertEquals(count1, count3); + } + + private String[] createTopicAndSendData(String topic) { + String[] data = { topic + "-1", topic + "-2", topic + "-3"}; + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); + return data; + } +} diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java new file mode 100644 index 000000000000..ab98f2bd90dd --- /dev/null +++ b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java @@ -0,0 +1,141 @@ +/* + * 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.v09; + +import java.io.Serializable; +import java.util.*; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.spark.streaming.kafka.v09.KafkaTestUtils; +import org.apache.spark.streaming.kafka.v09.KafkaUtils; +import scala.Tuple2; + +import kafka.serializer.StringDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.storage.StorageLevel; +import org.apache.spark.streaming.Duration; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +public class JavaKafkaStreamSuite implements Serializable { + private transient JavaStreamingContext ssc = null; + private transient Random random = new Random(); + private transient KafkaTestUtils kafkaTestUtils = null; + + @Before + public void setUp() { + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, new Duration(500)); + } + + @After + public void tearDown() { + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } + } + + @Test + public void testKafkaStream() throws InterruptedException { + String topic = "new_topic1_testKafkaStream"; + HashMap topics = new HashMap(); + topics.put(topic, 1); + + HashMap sent = new HashMap(); + sent.put("a", 5); + sent.put("b", 3); + sent.put("c", 10); + + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, sent); + + HashMap kafkaParams = new HashMap(); + kafkaParams.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaTestUtils.brokerAddress()); + kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, "test-consumer-" + random.nextInt(10000)); + kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + kafkaParams.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); + kafkaParams.put("spark.kafka.poll.time", "1000"); + + JavaPairDStream stream = KafkaUtils.createStream(ssc, + String.class, + String.class, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY_SER()); + + final Map result = Collections.synchronizedMap(new HashMap()); + + JavaDStream words = stream.map( + new Function, String>() { + @Override + public String call(Tuple2 tuple2) throws Exception { + return tuple2._2(); + } + } + ); + + words.countByValue().foreachRDD( + new Function, Void>() { + @Override + public Void call(JavaPairRDD rdd) throws Exception { + List> ret = rdd.collect(); + for (Tuple2 r : ret) { + if (result.containsKey(r._1())) { + result.put(r._1(), result.get(r._1()) + r._2()); + } else { + result.put(r._1(), r._2()); + } + } + + return null; + } + } + ); + + ssc.start(); + + long startTime = System.currentTimeMillis(); + boolean sizeMatches = false; + while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { + sizeMatches = sent.size() == result.size(); + Thread.sleep(200); + } + Assert.assertEquals(sent.size(), result.size()); + for (String k : sent.keySet()) { + Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); + } + } +} diff --git a/external/kafka-v09/src/test/resources/log4j.properties b/external/kafka-v09/src/test/resources/log4j.properties new file mode 100644 index 000000000000..75e3b53a093f --- /dev/null +++ b/external/kafka-v09/src/test/resources/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=target/unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.spark-project.jetty=WARN + diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala new file mode 100644 index 000000000000..027a383c915f --- /dev/null +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala @@ -0,0 +1,515 @@ +/* + * 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.v09 + +import java.io.File +import java.util.concurrent.atomic.AtomicLong + +import kafka.common.TopicAndPartition +import org.apache.kafka.clients.consumer.{ConsumerRecord, ConsumerConfig} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.scheduler.rate.RateEstimator +import org.apache.spark.streaming.scheduler.{StreamingListenerBatchCompleted, StreamingListenerBatchStarted, StreamingListenerBatchSubmitted, StreamingListener} +import org.apache.spark.streaming.{Time, Milliseconds, StreamingContext} +import org.apache.spark.util.Utils +import org.apache.spark.{SparkContext, SparkConf, Logging, SparkFunSuite} +import org.scalatest.concurrent.Eventually +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ +import scala.language.postfixOps + +class DirectKafkaStreamSuite + extends SparkFunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + + private var sc: SparkContext = _ + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + after { + if (ssc != null) { + ssc.stop() + sc = null + } + if (sc != null) { + sc.stop() + } + if (testDir != null) { + Utils.deleteRecursively(testDir) + } + } + + + test("basic stream receiving with multiple topics and earliest starting offset") { + val topics = Set("new_basic1", "new_basic2", "new_basic3") + val data = Map("a" -> 7, "b" -> 9) + topics.foreach { t => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + val totalSent = data.values.sum * topics.size + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "1000" + ) + + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, kafkaParams, topics) + } + + val allReceived = + new ArrayBuffer[(String, String)] with mutable.SynchronizedBuffer[(String, String)] + + // hold a reference to the current offset ranges, so it can be used downstream + var offsetRanges = Array[OffsetRange]() + + stream.transform { rdd => + // Get the offset ranges in the RDD + offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + rdd + }.foreachRDD { rdd => + for (o <- offsetRanges) { + log.info(s"${rdd.id} | ${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") + } + 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 = offsetRanges(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") + } + } + stream.foreachRDD { rdd => allReceived ++= rdd.collect() } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + allReceived.mkString("\n")) + } + ssc.stop() + } + + test("receiving from latest starting offset") { + val topic = "new_latest" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "latest", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "100" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestOffsets(Set(topicPartition)).right.get.get(topicPartition).get + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() > 3) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, kafkaParams, Set(topic)) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] + stream.map { + _._2 + }.foreachRDD { rdd => collectedData ++= rdd.collect() } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + } + + + test("creating stream by offset") { + val topic = "new_offset" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "latest", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "100" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestOffsets(Set(topicPartition)).right.get.get(topicPartition).get + } + + // Send some initial messages before starting context + kafkaTestUtils.sendMessages(topic, data) + eventually(timeout(10 seconds), interval(20 milliseconds)) { + assert(getLatestOffset() >= 10) + } + val offsetBeforeStart = getLatestOffset() + + // Setup context and kafka stream with largest offset + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, String]( + ssc, kafkaParams, Map(topicPartition -> 11L), + (m: ConsumerRecord[String, String]) => m.value()) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] + stream.foreachRDD { rdd => collectedData ++= rdd.collect() } + ssc.start() + val newData = Map("b" -> 10) + kafkaTestUtils.sendMessages(topic, newData) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + collectedData.contains("b") + } + assert(!collectedData.contains("a")) + } + + // Test to verify the offset ranges can be recovered from the checkpoints + test("offset recovery") { + val topic = "new_recovery" + kafkaTestUtils.createTopic(topic) + testDir = Utils.createTempDir() + + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "1000" + ) + + // Send data to Kafka and wait for it to be received + def sendDataAndWaitForReceive(data: Seq[Int]) { + val strings = data.map { + _.toString + } + kafkaTestUtils.sendMessages(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]( + 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() + } + + test("Direct Kafka stream report input information") { + val topic = "new_report-test" + val data = Map("a" -> 7, "b" -> 9) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + val totalSent = data.values.sum + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "1000" + ) + + import DirectKafkaStreamSuite._ + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val collector = new InputInfoCollector + ssc.addStreamingListener(collector) + + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String]( + ssc, kafkaParams, Set(topic)) + } + + val allReceived = + new ArrayBuffer[(String, String)] with mutable.SynchronizedBuffer[(String, String)] + + stream.foreachRDD { rdd => allReceived ++= rdd.collect() } + ssc.start() + eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { + assert(allReceived.size === totalSent, + "didn't get expected number of messages, messages:\n" + allReceived.mkString("\n")) + + // Calculate all the record number collected in the StreamingListener. + assert(collector.numRecordsSubmitted.get() === totalSent) + assert(collector.numRecordsStarted.get() === totalSent) + assert(collector.numRecordsCompleted.get() === totalSent) + } + ssc.stop() + } + + test("using rate controller") { + val topic = "new_backpressure" + val topicPartition = TopicAndPartition(topic, 0) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "1000" + ) + + val batchIntervalMilliseconds = 100 + val estimator = new ConstantEstimator(100) + val messageKeys = (1 to 200).map(_.toString) + val messages = messageKeys.map((_, 1)).toMap + + val sparkConf = new SparkConf() + // Safe, even with streaming, because we're using the direct API. + // Using 1 core is useful to make the test more predictable. + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val kafkaStream = withClue("Error creating direct stream") { + val kc = new KafkaCluster(kafkaParams) + val messageHandler = (mmd: ConsumerRecord[String, String]) => (mmd.key(), mmd.value()) + val m = kc.getEarliestOffsets(Set(topicPartition)).right.get + + new DirectKafkaInputDStream[String, String, (String, String)]( + ssc, kafkaParams, m, messageHandler) { + override protected[streaming] val rateController = + Some(new DirectKafkaRateController(id, estimator)) + } + } + + val collectedData = + new mutable.ArrayBuffer[Array[String]]() with mutable.SynchronizedBuffer[Array[String]] + + // Used for assertion failure messages. + def dataToString: String = + collectedData.map(_.mkString("[", ",", "]")).mkString("{", ", ", "}") + + // This is to collect the raw data received from Kafka + kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => + val data = rdd.map { + _._2 + }.collect() + collectedData += data + } + + ssc.start() + + // Try different rate limits. + // Send data to Kafka and wait for arrays of data to appear matching the rate. + Seq(100, 50, 20).foreach { rate => + collectedData.clear() // Empty this buffer on each pass. + estimator.updateRate(rate) // Set a new rate. + // Expect blocks of data equal to "rate", scaled by the interval length in secs. + val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) + kafkaTestUtils.sendMessages(topic, messages) + eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { + // Assert that rate estimator values are used to determine maxMessagesPerPartition. + // Funky "-" in message makes the complete assertion message read better. + assert(collectedData.exists(_.size == expectedSize), + s" - No arrays of size $expectedSize for rate $rate found in $dataToString") + } + } + + 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]() with mutable.SynchronizedBuffer[String] + @volatile var total = -1L + + class InputInfoCollector extends StreamingListener { + val numRecordsSubmitted = new AtomicLong(0L) + val numRecordsStarted = new AtomicLong(0L) + val numRecordsCompleted = new AtomicLong(0L) + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { + numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { + numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { + numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) + } + } + +} + +private[streaming] class ConstantEstimator(@volatile private var rate: Long) + extends RateEstimator { + + def updateRate(newRate: Long): Unit = { + rate = newRate + } + + def compute( + time: Long, + elements: Long, + processingDelay: Long, + schedulingDelay: Long): Option[Double] = Some(rate) +} + + diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala new file mode 100644 index 000000000000..980ad65fbe8c --- /dev/null +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala @@ -0,0 +1,67 @@ +/* + * 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.v09 + +import kafka.common.TopicAndPartition +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.spark.SparkFunSuite +import org.apache.spark.streaming.kafka.v09.KafkaTestUtils +import org.scalatest.BeforeAndAfterAll + +import scala.util.Random + +class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { + private val topic = "new_kcsuitetopic" + Random.nextInt(10000) + private val topicPartition = TopicAndPartition(topic, 0) + private var newKc: KafkaCluster[_, _] = null + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll() { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer" + ) + newKc = new KafkaCluster(kafkaParams) + } + + override def afterAll() { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + test("leader offset apis") { + val earliest = newKc.getEarliestOffsets(Set(topicPartition)).right.get + assert(earliest(topicPartition) === 0, "didn't get earliest") + + val latest = newKc.getLatestOffsets(Set(topicPartition)).right.get + assert(latest(topicPartition) === 1, "didn't get latest") + } + + +} diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala new file mode 100644 index 000000000000..5612a5876de2 --- /dev/null +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala @@ -0,0 +1,184 @@ +/* + * 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.v09 + +import kafka.common.TopicAndPartition +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} +import org.apache.spark._ +import org.apache.spark.streaming.kafka.v09.{HasOffsetRanges, KafkaTestUtils, KafkaUtils, OffsetRange} +import org.scalatest.BeforeAndAfterAll + +import scala.util.Random + +class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { + + private var kafkaTestUtils: KafkaTestUtils = _ + + private val sparkConf = new SparkConf().setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + private var sc: SparkContext = _ + + override def beforeAll { + sc = new SparkContext(sparkConf) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll { + if (sc != null) { + sc.stop + sc = null + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + test("basic usage") { + val topic = s"new_topicbasic-${Random.nextInt}" + kafkaTestUtils.createTopic(topic) + val messages = Array("the", "quick", "brown", "fox") + kafkaTestUtils.sendMessages(topic, messages) + + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "latest", + ConsumerConfig.GROUP_ID_CONFIG -> s"test-consumer-${Random.nextInt}", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "10000" + ) + + val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) + + val rdd = KafkaUtils.createRDD[String, String]( + sc, kafkaParams, offsetRanges) + + val received = rdd.map(_._2).collect.toSet + assert(received === messages.toSet) + + // size-related method optimizations return sane results + assert(rdd.count === messages.size) + assert(rdd.countApprox(0).getFinalValue.mean === messages.size) + assert(!rdd.isEmpty) + assert(rdd.take(1).size === 1) + assert(rdd.take(1).head._2 === messages.head) + assert(rdd.take(messages.size + 10).size === messages.size) + + val emptyRdd = KafkaUtils.createRDD[String, String]( + sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0))) + + assert(emptyRdd.isEmpty) + + // invalid offset ranges throw exceptions + val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) + intercept[SparkException] { + KafkaUtils.createRDD[String, String]( + sc, kafkaParams, badRanges) + } + } + + + 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 = s"new_topicboundary-${Random.nextInt}" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "latest", + ConsumerConfig.GROUP_ID_CONFIG -> s"test-consumer-${Random.nextInt}", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "1000" + ) + + val kc = new KafkaCluster(kafkaParams) + + // this is the "lots of messages" case + kafkaTestUtils.sendMessages(topic, sent) + val sentCount = sent.values.sum + + // rdd defined from leaders after sending messages, should get the number sent + val rdd = getRdd(kc, Set(topic)) + + assert(rdd.isDefined) + + val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges + val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum + + 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(rangesMap) + + // 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) + + assert(rdd2.isDefined) + assert(rdd2.get.count === 0, "got messages when there shouldn't be any") + + // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above + val rdd3 = getRdd(kc, Set(topic)) + // send lots of messages after rdd was defined, they shouldn't show up + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) + + assert(rdd3.isDefined) + assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message") + + } + + // get an rdd from the committed consumer offsets until the latest leader offsets, + private def getRdd(kc: KafkaCluster[_, _], topics: Set[String]) = { + val groupId = kc.kafkaParams("group.id") + def consumerOffsets(topicPartitions: Set[TopicAndPartition]) = { + kc.getCommittedOffsets(topicPartitions).right.toOption.orElse( + kc.getEarliestOffsets(topicPartitions).right.toOption.map { offs => + offs.map(kv => kv._1 -> kv._2) + } + ) + } + kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => + consumerOffsets(topicPartitions).flatMap { from => + kc.getLatestOffsets(topicPartitions).right.toOption.map { until => + val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) => + OffsetRange(tp.topic, tp.partition, fromOffset, until(tp)) + }.toArray + + KafkaUtils.createRDD[String, String, String]( + sc, kc.kafkaParams, offsetRanges, + (cr: ConsumerRecord[String, String]) => s"${cr.offset()} ${cr.value()}") + } + } + } + } +} diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala new file mode 100644 index 000000000000..558f57372c10 --- /dev/null +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.v09 + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.kafka.v09.{KafkaTestUtils, KafkaUtils} +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually + +import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random + +class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfterAll { + private var ssc: StreamingContext = _ + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll(): Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + } + + override def afterAll(): Unit = { + if (ssc != null) { + ssc.stop() + ssc = null + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + test("Kafka input stream") { + val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) + ssc = new StreamingContext(sparkConf, Milliseconds(500)) + val topic = "new_topic_NewKafkaStreamSuite" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, sent) + + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", + ConsumerConfig.GROUP_ID_CONFIG -> s"test-consumer-${Random.nextInt(10000)}", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "100" + ) + + val stream = KafkaUtils.createStream[String, String]( + ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() with mutable.SynchronizedMap[String, Long] + stream.map(_._2).countByValue().foreachRDD { r => + val ret = r.collect() + ret.toMap.foreach { kv => + val count = result.getOrElseUpdate(kv._1, 0) + kv._2 + result.put(kv._1, count) + } + } + + ssc.start() + + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { + assert(sent === result) + } + } +} diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala new file mode 100644 index 000000000000..139f8573a0e7 --- /dev/null +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala @@ -0,0 +1,158 @@ +/* + * 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.v09 + +import java.io.File + +import kafka.common.TopicAndPartition +import kafka.utils.ZkUtils +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.util.Utils +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.scalatest.concurrent.Eventually +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} + +import scala.collection.mutable +import scala.concurrent.duration._ +import scala.language.postfixOps +import scala.util.Random + +class ReliableKafkaStreamSuite extends SparkFunSuite +with BeforeAndAfterAll with BeforeAndAfter with Eventually { + + private val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.receiver.writeAheadLog.enable", "true") + private val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + + private var kafkaTestUtils: KafkaTestUtils = _ + + private var groupId: String = _ + private var kafkaParams: Map[String, String] = _ + private var ssc: StreamingContext = _ + private var tempDirectory: File = null + private var zkUtils: ZkUtils = null + + private var kc: KafkaCluster[_, _] = null + + override def beforeAll() : Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + groupId = s"new_test-consumer-${Random.nextInt(10000)}" + + kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", + ConsumerConfig.GROUP_ID_CONFIG -> groupId, + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "100" + ) + + kc = new KafkaCluster(kafkaParams) + + tempDirectory = Utils.createTempDir() + zkUtils = ZkUtils(kafkaTestUtils.zookeeperClient, false) + } + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDirectory) + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + before { + ssc = new StreamingContext(sparkConf, Milliseconds(500)) + ssc.checkpoint(tempDirectory.getAbsolutePath) + } + + after { + if (ssc != null) { + ssc.stop() + ssc = null + } + } + + test("Reliable Kafka input stream with single topic") { + val topic = "test-topic" + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + // Verify whether the offset of this group/topic/partition is 0 before starting. + assert(getCommitOffset(topic, 0) === None) + + val stream = KafkaUtils.createStream[String, String]( + ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) + val result = new mutable.HashMap[String, Long]() + stream.map { case (k, v) => v }.foreachRDD { r => + val ret = r.collect() + ret.foreach { v => + val count = result.getOrElseUpdate(v, 0) + 1 + result.put(v, count) + } + } + ssc.start() + + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { + // A basic process verification for ReliableKafkaReceiver. + // Verify whether received message number is equal to the sent message number. + assert(data.size === result.size) + // Verify whether each message is the same as the data to be verified. + data.keys.foreach { k => assert(data(k) === result(k).toInt) } + // Verify the offset number whether it is equal to the total message number. + assert(getCommitOffset(topic, 0) === Some(29L)) + } + } + + test("Reliable Kafka input stream with multiple topics") { + val topics = Map("new_topic1" -> 1, "new_topic2" -> 1, "new_topic3" -> 1) + topics.foreach { case (t, _) => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) + } + + // Before started, verify all the group/topic/partition offsets are 0. + // topics.foreach { case (t, _) => assert(getCommitOffset(t, 0) === None) } + + // Consuming all the data sent to the broker which will potential commit the offsets internally. + val stream = KafkaUtils.createStream[String, String]( + ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) + stream.foreachRDD(_ => Unit) + ssc.start() + + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { + // Verify the offset for each group/topic to see whether they are equal to the expected one. + topics.foreach { case (t, _) => assert(getCommitOffset(t, 0) === Some(29L)) } + } + } + + + /** Getting partition offset from Zookeeper. */ + private def getCommitOffset(topic: String, partition: Int): Option[Long] = { + kc.getCommittedOffset(TopicAndPartition(topic, partition)).right.toOption + } +} diff --git a/pom.xml b/pom.xml index 234fd5dea1a6..f180fead7d56 100644 --- a/pom.xml +++ b/pom.xml @@ -112,6 +112,7 @@ repl launcher external/kafka + external/kafka-v09 external/kafka-assembly From 328379e722124d62666f1ded84636d4711f6c012 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Tue, 15 Dec 2015 12:22:07 +0200 Subject: [PATCH 02/16] Refactor --- .../examples/streaming/KafkaWordCount.scala | 49 +- .../streaming/v09DirectKafkaWordCount.scala | 46 +- external/kafka-v09/pom.xml | 18 + .../kafka/v09/DirectKafkaInputDStream.scala | 61 +- .../streaming/kafka/v09/KafkaCluster.scala | 12 +- .../kafka/v09/KafkaInputDStream.scala | 16 +- .../spark/streaming/kafka/v09/KafkaRDD.scala | 94 ++-- .../kafka/v09/KafkaRDDPartition.scala | 20 +- .../streaming/kafka/v09/KafkaTestUtils.scala | 7 +- .../streaming/kafka/v09/KafkaUtils.scala | 530 ++++++++---------- .../kafka/v09/ReliableKafkaReceiver.scala | 86 ++- .../kafka/v09/DirectKafkaStreamSuite.scala | 63 +-- .../kafka/v09/KafkaClusterSuite.scala | 5 +- .../streaming/kafka/v09/KafkaRDDSuite.scala | 18 +- .../kafka/v09/KafkaStreamSuite.scala | 8 +- .../kafka/v09/ReliableKafkaStreamSuite.scala | 23 +- 16 files changed, 508 insertions(+), 548 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 6fd331bd3228..6115fd28f073 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -20,7 +20,6 @@ package org.apache.spark.examples.streaming import java.util.HashMap -import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer, ProducerRecord} import org.apache.spark.streaming._ @@ -28,18 +27,18 @@ import org.apache.spark.streaming.kafka._ import org.apache.spark.SparkConf /** - * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: KafkaWordCount - * is a list of one or more zookeeper servers that make quorum - * is the name of kafka consumer group - * is a list of one or more kafka topics to consume from - * is the number of threads the kafka consumer should use - * - * Example: - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.KafkaWordCount zoo01,zoo02,zoo03 \ - * my-consumer-group topic1,topic2 1` - */ + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: KafkaWordCount + * is a list of one or more zookeeper servers that make quorum + * is the name of kafka consumer group + * is a list of one or more kafka topics to consume from + * is the number of threads the kafka consumer should use + * + * Example: + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.KafkaWordCount zoo01,zoo02,zoo03 \ + * my-consumer-group topic1,topic2 1` + */ object KafkaWordCount { def main(args: Array[String]) { if (args.length < 4) { @@ -67,18 +66,18 @@ object KafkaWordCount { } /** - * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: NewKafkaWordCount - * is a list of one or more zookeeper servers that make quorum - * is the name of kafka consumer group - * is a list of one or more kafka topics to consume from - * is the number of threads the kafka consumer should use - * - * Example: - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.NewKafkaWordCount broker1,broker2,broker3 \ - * my-consumer-group topic1,topic2 1` - */ + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: v09KafkaWordCount + * is a list of one or more brokers servers + * is the name of kafka consumer group + * is a list of one or more kafka topics to consume from + * is the number of threads the kafka consumer should use + * + * Example: + * `$ bin/run-example \ + * org.apache.spark.examples.streaming.v09KafkaWordCount broker01,broker02,broker03 \ + * my-consumer-group topic1,topic2 1` + */ object v09KafkaWordCount { def main(args: Array[String]) { import org.apache.spark.streaming.kafka.v09._ diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala index 8d5485992718..7dca188dd7a1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/v09DirectKafkaWordCount.scala @@ -27,15 +27,22 @@ import org.apache.spark.streaming.kafka.v09._ import org.apache.spark.SparkConf /** - * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: v09DirectKafkaWordCount - * is a list of one or more Kafka brokers - * is a list of one or more kafka topics to consume from - * - * Example: - * $ bin/run-example streaming.v09DirectKafkaWordCount broker1-host:port,broker2-host:port \ - * topic1,topic2 - */ + * Consumes messages from one or more topics in Kafka and does wordcount. + * Usage: v09DirectKafkaWordCount + * is a list of one or more Kafka brokers + * is a list of one or more kafka topics to consume from + * is the name of kafka consumer group + * What to do when there is no initial offset in Kafka or + * if the current offset does not exist any more on the server + * earliest: automatically reset the offset to the earliest offset + * latest: automatically reset the offset to the latest offset + * is the time interval at which streaming data will be divided into batches + * is time, in milliseconds, spent waiting in Kafka consumer poll + * if data is not available + * Example: + * $ bin/run-example streaming.v09DirectKafkaWordCount broker1-host:port,broker2-host:port \ + * topic1,topic2 my-consumer-group latest batch-interval pollTimeout + */ object v09DirectKafkaWordCount { def main(args: Array[String]) { if (args.length < 2) { @@ -43,10 +50,18 @@ object v09DirectKafkaWordCount { |Usage: v09DirectKafkaWordCount | is a list of one or more Kafka brokers | is a list of one or more kafka topics to consume from - | comsumer group id - | latest (default) or earliest - | time in seconds - | time in milliseconds + | is the name of kafka consumer group + | What to do when there is no initial offset + | in Kafka or if the current offset does not exist + | any more on the server + | earliest: automatically reset the offset + | to the earliest offset + | latest: automatically reset the offset + | to the latest offset + | is the time interval at which + | streaming data will be divided into batches + | is time, in milliseconds, spent waiting in + | Kafka consumer poll if data is not available | """.stripMargin) System.exit(1) @@ -59,7 +74,6 @@ object v09DirectKafkaWordCount { // Create context with 2 second batch interval val sparkConf = new SparkConf().setAppName("v09DirectKafkaWordCount") val ssc = new StreamingContext(sparkConf, Seconds(batchInterval.toInt)) - val reset = if (offsetReset.isEmpty) "latest" else offsetReset // Create direct kafka stream with brokers and topics val topicsSet = topics.split(",").toSet @@ -70,9 +84,9 @@ object v09DirectKafkaWordCount { "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> reset, + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> offsetReset, ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG -> "false", - "spark.kafka.poll.time" -> pollTimeout) + "spark.kafka.poll.time" -> pollTimeout) val messages = KafkaUtils.createDirectStream[String, String](ssc, kafkaParams, topicsSet) // Get the lines, split them into words, count the words and print diff --git a/external/kafka-v09/pom.xml b/external/kafka-v09/pom.xml index a09b54466690..e30bd9289464 100644 --- a/external/kafka-v09/pom.xml +++ b/external/kafka-v09/pom.xml @@ -1,3 +1,21 @@ + + + 4.0.0 diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala index 4416f38c235d..dac33a097247 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala @@ -29,33 +29,32 @@ import scala.collection.mutable import scala.reflect.ClassTag /** - * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where - * each given Kafka topic/partition corresponds to an RDD partition. - * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number - * of messages - * per second that each '''partition''' will accept. - * Starting offsets are specified in advance, - * and this DStream is not responsible for committing offsets, - * so that you can control exactly-once semantics. - * For an easy interface to Kafka-managed offsets, - * see {@link org.apache.spark.streaming.kafka.KafkaCluster} - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - */ + * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where + * each given Kafka topic/partition corresponds to an RDD partition. + * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number + * of messages + * per second that each '''partition''' will accept. + * Starting offsets are specified in advance, + * and this DStream is not responsible for committing offsets, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * configuration parameters. + * Requires "metadata.broker.list" or "bootstrap.servers" to be set + * with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + */ private[streaming] class DirectKafkaInputDStream[ -K: ClassTag, -V: ClassTag, -R: ClassTag]( - @transient ssc_ : StreamingContext, - val kafkaParams: Map[String, String], - val fromOffsets: Map[TopicAndPartition, Long], - messageHandler: ConsumerRecord[K, V] => R - ) extends InputDStream[R](ssc_) with Logging { + K: ClassTag, + V: ClassTag, + R: ClassTag]( + @transient ssc_ : StreamingContext, + val kafkaParams: Map[String, String], + val fromOffsets: Map[TopicAndPartition, Long], + messageHandler: ConsumerRecord[K, V] => R + ) extends InputDStream[R](ssc_) with Logging { val maxRetries = context.sparkContext.getConf.getInt( "spark.streaming.kafka.maxRetries", 1) @@ -68,8 +67,8 @@ R: ClassTag]( /** - * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. - */ + * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. + */ override protected[streaming] val rateController: Option[RateController] = { if (RateController.isBackPressureEnabled(ssc.conf)) { Some(new DirectKafkaRateController(id, @@ -83,6 +82,7 @@ R: ClassTag]( private val maxRateLimitPerPartition: Int = context.sparkContext.getConf.getInt( "spark.streaming.kafka.maxRatePerPartition", 0) + protected def maxMessagesPerPartition: Option[Long] = { val estimatedRateLimit = rateController.map(_.getLatestRate().toInt) val numPartitions = currentOffsets.keys.size @@ -174,7 +174,7 @@ R: ClassTag]( } } - override def cleanup(time: Time) { } + override def cleanup(time: Time) {} override def restore() { // this is assuming that the topics don't change during execution, which is true currently @@ -189,10 +189,11 @@ R: ClassTag]( } /** - * A RateController to retrieve the rate from RateEstimator. - */ + * A RateController to retrieve the rate from RateEstimator. + */ private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) extends RateController(id, estimator) { override def publish(rate: Long): Unit = () } + } diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala index 9980d69ca597..c71bacbfbde8 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala @@ -32,15 +32,15 @@ import scala.reflect._ import scala.util.control.NonFatal /** - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form - */ + * @param kafkaParams Kafka + * configuration parameters. + * Requires "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form + */ private[spark] class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String]) extends Serializable { + import KafkaCluster.{Err, SeekType, toTopicPart} def getLatestOffsets(topicPartitions: Set[TopicAndPartition]): diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala index a7b251ef7231..30f10527cabe 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala @@ -31,14 +31,14 @@ import scala.collection.Map import scala.reflect.ClassTag /** - * Input stream that pulls messages from a Kafka Broker. - * - * @param kafkaParams Map of kafka configuration parameters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - */ + * Input stream that pulls messages from a Kafka Broker. + * + * @param kafkaParams Map of kafka configuration parameters. + * See: http://kafka.apache.org/configuration.html + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread. + * @param storageLevel RDD storage level. + */ private[streaming] class KafkaInputDStream[ K: ClassTag, diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala index 00042409f918..5a454218dd83 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala @@ -17,53 +17,53 @@ package org.apache.spark.streaming.kafka.v09 -import java.util.{Collections, Properties} +import java.util.{ Collections, Properties } import kafka.common.TopicAndPartition -import org.apache.kafka.clients.consumer.{ConsumerRecord, KafkaConsumer} +import org.apache.kafka.clients.consumer.{ ConsumerRecord, KafkaConsumer } import org.apache.kafka.common.TopicPartition -import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.partial.{ BoundedDouble, PartialResult } import org.apache.spark.rdd.RDD import org.apache.spark.streaming.kafka.v09.KafkaCluster.toTopicPart import org.apache.spark.util.NextIterator -import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} +import org.apache.spark.{ Logging, Partition, SparkContext, TaskContext } import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag /** - * A batch-oriented interface for consuming from Kafka. - * Starting and ending offsets are specified in advance, - * so that you can control exactly-once semantics. - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set - * with Kafka broker(s) specified in host1:port1,host2:port2 form. - * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD - */ + * A batch-oriented interface for consuming from Kafka. + * Starting and ending offsets are specified in advance, + * so that you can control exactly-once semantics. + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + */ private[kafka] -class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark]( - sc: SparkContext, - kafkaParams: Map[String, String], - val offsetRanges: Array[OffsetRange], - messageHandler: ConsumerRecord[K, V] => R - ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { +class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( + sc: SparkContext, + kafkaParams: Map[String, String], + val offsetRanges: Array[OffsetRange], + messageHandler: ConsumerRecord[K, V] => R + ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { private val KAFKA_DEFAULT_POLL_TIME: String = "0" private val pollTime = kafkaParams.get("spark.kafka.poll.time") .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt override def getPartitions: Array[Partition] = { - offsetRanges.zipWithIndex.map { case (o, i) => - new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) + offsetRanges.zipWithIndex.map { + case (o, i) => + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) }.toArray } override def count(): Long = offsetRanges.map(_.count).sum override def countApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[BoundedDouble] = { + timeout: Long, + confidence: Double = 0.95): PartialResult[BoundedDouble] = { val c = count new PartialResult(new BoundedDouble(c, 1.0, c, c), true) } @@ -127,10 +127,10 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark]( } private class KafkaRDDIterator( - part: KafkaRDDPartition, - context: TaskContext) extends NextIterator[R] { + part: KafkaRDDPartition, + context: TaskContext) extends NextIterator[R] { - context.addTaskCompletionListener{ context => closeIfNeeded() } + context.addTaskCompletionListener { context => closeIfNeeded() } log.info(s"Computing topic ${part.topic}, partition ${part.partition} " + s"offsets ${part.fromOffset} -> ${part.untilOffset}") @@ -173,34 +173,32 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark]( } } } + } private[kafka] object KafkaRDD { /** - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the batch - * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) - * ending point of the batch - */ - def apply[ - K: ClassTag, - V: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - untilOffsets: Map[TopicAndPartition, Long], - messageHandler: ConsumerRecord[K, V] => R - ): KafkaRDD[K, V, R] = { - val offsetRanges = fromOffsets.map { case (tp, fo) => - val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo) + * @param kafkaParams Kafka + * configuration parameters. + * Requires "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the batch + * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) + * ending point of the batch + */ + def apply[K: ClassTag, V: ClassTag, R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + untilOffsets: Map[TopicAndPartition, Long], + messageHandler: ConsumerRecord[K, V] => R): KafkaRDD[K, V, R] = { + val offsetRanges = fromOffsets.map { + case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo) }.toArray new KafkaRDD[K, V, R](sc, kafkaParams, offsetRanges, messageHandler) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala index df1e4be5c18f..0df2a8a3fd8f 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala @@ -20,18 +20,18 @@ package org.apache.spark.streaming.kafka.v09 import org.apache.spark.Partition /** @param topic kafka topic name - * @param partition kafka partition id - * @param fromOffset inclusive starting offset - * @param untilOffset exclusive ending offset - */ + * @param partition kafka partition id + * @param fromOffset inclusive starting offset + * @param untilOffset exclusive ending offset + */ private[kafka] class KafkaRDDPartition( - val index: Int, - val topic: String, - val partition: Int, - val fromOffset: Long, - val untilOffset: Long - ) extends Partition { + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long + ) extends Partition { /** Number of messages this partition refers to */ def count(): Long = untilOffset - fromOffset } diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala index 4f30d0d42c19..c5baaeca9e18 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaTestUtils.scala @@ -178,7 +178,9 @@ private[kafka] class KafkaTestUtils extends Logging { /** Send the array of messages to the Kafka broker */ def sendMessages(topic: String, messages: Array[String]): Unit = { producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) - producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) + producer.send(messages.map { + new KeyedMessage[String, String](topic, _) + }: _*) producer.close() producer = null } @@ -240,7 +242,7 @@ private[kafka] class KafkaTestUtils extends Logging { case Some(partitionState) => val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr - zkUtils.getLeaderForPartition(topic, partition).isDefined && + zkUtils.getLeaderForPartition(topic, partition).isDefined && Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && leaderAndInSyncReplicas.isr.size >= 1 @@ -273,5 +275,6 @@ private[kafka] class KafkaTestUtils extends Logging { Utils.deleteRecursively(logDir) } } + } diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala index 2a228659f80d..67e6690d93ef 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala @@ -46,36 +46,34 @@ import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputD object KafkaUtils { /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param ssc StreamingContext object - * @param servers Broker servers (for Kafka 0.9) (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - */ + * Create an input stream that pulls messages from Kafka Brokers. + * @param ssc StreamingContext object + * @param servers Broker servers (for Kafka 0.9) (hostname:port,hostname:port,..) + * @param groupId The group id for this consumer + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread + * @param storageLevel Storage level to use for storing the received objects + * (default: StorageLevel.MEMORY_AND_DISK_SER_2) + */ def createStream( - ssc: StreamingContext, - servers: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[(String, String)] = { - - val kafkaParams = Map[String, String]( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> servers, - ConsumerConfig.GROUP_ID_CONFIG -> groupId, - ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG -> "5000", - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + ssc: StreamingContext, + servers: String, + groupId: String, + topics: Map[String, Int], + storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 + ): ReceiverInputDStream[(String, String)] = { + val kafkaParams = Map[String, String]( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> servers, + ConsumerConfig.GROUP_ID_CONFIG -> groupId, + ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG -> "5000", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "1000" - ) - createStream[String, String]( - ssc, kafkaParams, topics, storageLevel) - + "spark.kafka.poll.time" -> "1000" + ) + createStream[String, String]( + ssc, kafkaParams, topics, storageLevel) } /** @@ -96,54 +94,16 @@ object KafkaUtils { new KafkaInputDStream[K, V](ssc, kafkaParams, topics, walEnabled, storageLevel) } - /** - * Create an input stream that pulls messages from Kafka Brokers. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - */ - def createStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt] - ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*)) - } - /** * Create an input stream that pulls messages from Kafka Brokers. * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. + * @param keyTypeClass Key type of DStream + * @param valueTypeClass value type of Dstream + * @param kafkaParams Map of kafka configuration parameters + * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed + * in its own thread * @param storageLevel RDD storage level. */ - def createStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param jssc JavaStreamingContext object - * @param keyTypeClass Key type of DStream - * @param valueTypeClass value type of Dstream - * @param kafkaParams Map of kafka configuration parameters - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel RDD storage level. - */ def createStream[K, V]( jssc: JavaStreamingContext, keyTypeClass: Class[K], @@ -183,13 +143,11 @@ object KafkaUtils { } - def createRDD[ - K: ClassTag, - V: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange] - ): RDD[(K, V)] = sc.withScope { + def createRDD[K: ClassTag, V: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange] + ): RDD[(K, V)] = sc.withScope { val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) val kc = new KafkaCluster[K, V](kafkaParams) checkOffsets(kc, offsetRanges) @@ -197,28 +155,25 @@ object KafkaUtils { } /** - * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you - * specify the Kafka leader to connect to (to optimize fetching) and access the message as well - * as the metadata. - * - * @param sc SparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @param messageHandler Function for translating each message and metadata into the desired type - */ - def createRDD[ - K: ClassTag, - V: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange], - messageHandler: ConsumerRecord[K, V] => R - ): RDD[R] = sc.withScope { + * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param sc SparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param messageHandler Function for translating each message and metadata into the desired type + */ + def createRDD[K: ClassTag, V: ClassTag, R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange], + messageHandler: ConsumerRecord[K, V] => R + ): RDD[R] = sc.withScope { val kc = new KafkaCluster[K, V](kafkaParams) val cleanedHandler = sc.clean(messageHandler) checkOffsets(kc, offsetRanges) @@ -226,22 +181,22 @@ object KafkaUtils { } /** - * Create a RDD from Kafka using offset ranges for each topic and partition. - * - * @param jsc JavaSparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "bootstrap.servers" - * specified in host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - */ + * Create a RDD from Kafka using offset ranges for each topic and partition. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * specified in host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + */ def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - kafkaParams: JMap[String, String], - offsetRanges: Array[OffsetRange] - ): JavaPairRDD[K, V] = jsc.sc.withScope { + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange] + ): JavaPairRDD[K, V] = jsc.sc.withScope { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) new JavaPairRDD(createRDD[K, V]( @@ -249,27 +204,27 @@ object KafkaUtils { } /** - * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you - * specify the Kafka leader to connect to (to optimize fetching) and access the message as well - * as the metadata. - * - * @param jsc JavaSparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "bootstrap.servers" - * specified in host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @param messageHandler Function for translating each message and metadata into the desired type - */ + * Create a RDD from Kafka using offset ranges for each topic and partition. This allows you + * specify the Kafka leader to connect to (to optimize fetching) and access the message as well + * as the metadata. + * + * @param jsc JavaSparkContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * specified in host1:port1,host2:port2 form. + * @param offsetRanges Each OffsetRange in the batch corresponds to a + * range of offsets for a given Kafka topic/partition + * @param messageHandler Function for translating each message and metadata into the desired type + */ def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - recordClass: Class[R], - kafkaParams: JMap[String, String], - offsetRanges: Array[OffsetRange], - messageHandler: JFunction[ConsumerRecord[K, V], R] - ): JavaRDD[R] = jsc.sc.withScope { + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + recordClass: Class[R], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange], + messageHandler: JFunction[ConsumerRecord[K, V], R] + ): JavaRDD[R] = jsc.sc.withScope { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) @@ -277,96 +232,89 @@ object KafkaUtils { jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, messageHandler.call _) } - -/** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param ssc StreamingContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler Function for translating each message and metadata into the desired type - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag, - R: ClassTag] ( - ssc: StreamingContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - messageHandler: ConsumerRecord[K, V] => R - ): InputDStream[R] = { + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers) specified in + * host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + */ + def createDirectStream[K: ClassTag, V: ClassTag, R: ClassTag]( + ssc: StreamingContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + messageHandler: ConsumerRecord[K, V] => R + ): InputDStream[R] = { val cleanedHandler = ssc.sc.clean(messageHandler) new DirectKafkaInputDStream[K, V, R]( ssc, kafkaParams, fromOffsets, messageHandler) } /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param ssc StreamingContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" - * to determine where the stream starts (defaults to "largest") - * @param topics Names of the topics to consume - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag] ( + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param ssc StreamingContext object + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set to + * "earliest" or "latest" to determine where the stream starts + * (defaults to "latest") + * @param topics Names of the topics to consume + */ + def createDirectStream[K: ClassTag, V: ClassTag]( ssc: StreamingContext, kafkaParams: Map[String, String], topics: Set[String] - ): InputDStream[(K, V)] = { + ): InputDStream[(K, V)] = { val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) val kc = new KafkaCluster[K, V](kafkaParams) val reset = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).map(_.toLowerCase) val fromOffsets = if (reset == Some("earliest")) { - kc.getEarliestOffsets(kc.getPartitions(topics).right.get).right.get + kc.getEarliestOffsets(kc.getPartitions(topics).right.get).right.get } else { - kc.getLatestOffsets(kc.getPartitions(topics).right.get).right.get + kc.getLatestOffsets(kc.getPartitions(topics).right.get).right.get } kc.close() @@ -375,46 +323,45 @@ object KafkaUtils { ssc, kafkaParams, fromOffsets, messageHandler) } -/** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param jssc JavaStreamingContext object - * @param keyClass Class of the keys in the Kafka records - * @param valueClass Class of the values in the Kafka records - * @param recordClass Class of the records in DStream - * @param kafkaParams Kafka - * configuration parameters. Requires "bootstrap.servers" - * specified in host1:port1,host2:port2 form. - * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler Function for translating each message and metadata into the desired type - */ + /** + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param recordClass Class of the records in DStream + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * specified in host1:port1,host2:port2 form. + * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) + * starting point of the stream + * @param messageHandler Function for translating each message and metadata into the desired type + */ def createDirectStream[K, V, R]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - recordClass: Class[R], - kafkaParams: JMap[String, String], - fromOffsets: JMap[TopicAndPartition, JLong], - messageHandler: JFunction[ConsumerRecord[K, V], R] + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + recordClass: Class[R], + kafkaParams: JMap[String, String], + fromOffsets: JMap[TopicAndPartition, JLong], + messageHandler: JFunction[ConsumerRecord[K, V], R] ): JavaInputDStream[R] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) @@ -423,50 +370,52 @@ object KafkaUtils { createDirectStream[K, V, R]( jssc.ssc, Map(kafkaParams.asScala.toSeq: _*), - Map(fromOffsets.asScala.mapValues { _.longValue() }.toSeq: _*), + Map(fromOffsets.asScala.mapValues { + _.longValue() + }.toSeq: _*), cleanedHandler ) } /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the [[StreamingContext]]. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param jssc JavaStreamingContext object - * @param keyClass Class of the keys in the Kafka records - * @param valueClass Class of the values in the Kafka records - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" - * to determine where the stream starts (defaults to "largest") - * @param topics Names of the topics to consume - */ + * Create an input stream that directly pulls messages from Kafka Brokers + * without using any receiver. This stream can guarantee that each message + * from Kafka is included in transformations exactly once (see points below). + * + * Points to note: + * - No receivers: This stream does not use any receiver. It directly queries Kafka + * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked + * by the stream itself. + * You can access the offsets used in each batch from the generated RDDs (see + * [[org.apache.spark.streaming.kafka.v09.HasOffsetRanges]]). + * - Failure Recovery: To recover from driver failures, you have to enable checkpointing + * in the [[StreamingContext]]. The information on consumed offset can be + * recovered from the checkpoint. See the programming guide for details (constraints, etc.). + * - End-to-end semantics: This stream ensures that every records is effectively received and + * transformed exactly once, but gives no guarantees on whether the transformed data are + * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure + * that the output operation is idempotent, or use transactions to output records atomically. + * See the programming guide for more details. + * + * @param jssc JavaStreamingContext object + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param kafkaParams Kafka + * configuration parameters. Requires "bootstrap.servers" + * to be set with Kafka broker(s) (NOT zookeeper servers), specified in + * host1:port1,host2:port2 form. + * If not starting from a checkpoint, "auto.offset.reset" may be set + * to "latest" or "earliest" to determine where the stream starts + * (defaults to "latest") + * @param topics Names of the topics to consume + */ def createDirectStream[K, V]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - kafkaParams: JMap[String, String], - topics: JSet[String] - ): JavaPairInputDStream[K, V] = { + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + kafkaParams: JMap[String, String], + topics: JSet[String] + ): JavaPairInputDStream[K, V] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) createDirectStream[K, V]( @@ -476,8 +425,12 @@ object KafkaUtils { ) } - def createOffsetRange(topic: String, partition: JInt, fromOffset: JLong, untilOffset: JLong - ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) + def createOffsetRange( + topic: String, + partition: JInt, + fromOffset: JLong, + untilOffset: JLong + ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = TopicAndPartition(topic, partition) @@ -535,4 +488,5 @@ private object KafkaUtilsPythonHelper { } } } + } diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala index 3946770dac91..dbaa62dd4f9d 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala @@ -35,26 +35,23 @@ import scala.collection.{Map, mutable} import scala.reflect.ClassTag /** - * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager - * without loss. - * It is turned off by default and will be enabled when - * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver - * is that this receiver manages topic-partition/offset itself and updates the offset information - * after data is reliably stored as write-ahead log. Offsets will only be updated when data is - * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated. - * - * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset - * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams - * will not take effect. - */ + * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager + * without loss. + * It is turned off by default and will be enabled when + * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver + * is that this receiver manages topic-partition/offset itself and updates the offset information + * after data is reliably stored as write-ahead log. Offsets will only be updated when data is + * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated. + * + * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset + * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams + * will not take effect. + */ private[streaming] -class ReliableKafkaReceiver[ -K: ClassTag, -V: ClassTag]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel) - extends Receiver[(K, V)](storageLevel) with Logging { +class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( + kafkaParams: Map[String, String], + topics: Map[String, Int], + storageLevel: StorageLevel) extends Receiver[(K, V)](storageLevel) with Logging { private val groupId = kafkaParams("group.id") private val AUTO_OFFSET_COMMIT = ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG @@ -68,35 +65,34 @@ V: ClassTag]( private val KAFKA_DEFAULT_POLL_TIME: String = "0" private val pollTime = kafkaParams.get("spark.kafka.poll.time") - .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt + .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt /** - * A HashMap to manage the offset for each topic/partition, this HashMap is called in - * synchronized block, so mutable HashMap will not meet concurrency issue. - */ + * A HashMap to manage the offset for each topic/partition, this HashMap is called in + * synchronized block, so mutable HashMap will not meet concurrency issue. + */ private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null /** A concurrent HashMap to store the stream block id and related offset snapshot. */ private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null /** - * Manage the BlockGenerator in receiver itself for better managing block store and offset - * commit. - */ + * Manage the BlockGenerator in receiver itself for better managing block store and offset + * commit. + */ private var blockGenerator: BlockGenerator = null /** Thread pool running the handlers for receiving message from multiple topics and partitions. */ private var messageHandlerThreadPool: ThreadPoolExecutor = null private var topicAndPartitionConsumerMap: - mutable.HashMap[TopicAndPartition, KafkaConsumer[K, V]] = null + mutable.HashMap[TopicAndPartition, KafkaConsumer[K, V]] = null private var consumerAndLockMap: - mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] = null + mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] = null override def onStart(): Unit = { logInfo(s"Starting Kafka Consumer Stream with group: $groupId") - logWarning("[!] -> Starting 0.9 ReliableKafkaReceiver") // Initialize the topic-partition / offset hash map. topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] @@ -111,7 +107,7 @@ V: ClassTag]( if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + - "otherwise we will manually set it to false to turn off auto offset commit in Kafka") + "otherwise we will manually set it to false to turn off auto offset commit in Kafka") } props = new Properties() @@ -171,8 +167,7 @@ V: ClassTag]( } /** Store a Kafka message and the associated metadata as a tuple. */ - private def storeMessageAndMetadata( - msgAndMetadata: MessageAndMetadata[K, V]): Unit = { + private def storeMessageAndMetadata(msgAndMetadata: MessageAndMetadata[K, V]): Unit = { val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) val data = (msgAndMetadata.key, msgAndMetadata.message) val metadata = (topicAndPartition, msgAndMetadata.offset) @@ -192,9 +187,9 @@ V: ClassTag]( } /** - * Remember the current offsets for each topic and partition. This is called when a block is - * generated. - */ + * Remember the current offsets for each topic and partition. This is called when a block is + * generated. + */ private def rememberBlockOffsets(blockId: StreamBlockId): Unit = { // Get a snapshot of current offset map and store with related block id. val offsetSnapshot = topicPartitionOffsetMap.toMap @@ -203,12 +198,13 @@ V: ClassTag]( } /** - * Store the ready-to-be-stored block and commit the related offsets to zookeeper. This method - * will try a fixed number of times to push the block. If the push fails, - * the receiver is stopped. - */ - private def storeBlockAndCommitOffset(blockId: StreamBlockId, - arrayBuffer: mutable.ArrayBuffer[_]): Unit = { + * Store the ready-to-be-stored block and commit the related offsets to Kafka. This method + * will try a fixed number of times to push the block. If the push fails, + * the receiver is stopped. + */ + private def storeBlockAndCommitOffset( + blockId: StreamBlockId, + arrayBuffer: mutable.ArrayBuffer[_]): Unit = { var count = 0 var pushed = false var exception: Exception = null @@ -231,13 +227,13 @@ V: ClassTag]( } /** - * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's - * metadata schema in Zookeeper. - */ + * Commit the offset of Kafka's topic/partition + */ private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = { val offsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() for ((topicAndPart, offset) <- offsetMap) { - val kafkaConsumer = topicAndPartitionConsumerMap.get(topicAndPart).get // todo: remove get + val kafkaConsumer = topicAndPartitionConsumerMap.getOrElse(topicAndPart, + throw new RuntimeException(s"Failed to get consumer for $topicAndPart")) val topicPartition = new TopicPartition(topicAndPart.topic, topicAndPart.partition) val offsetAndMetadata = new OffsetAndMetadata(offset) diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala index 027a383c915f..f07d635850a4 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala @@ -21,16 +21,16 @@ import java.io.File import java.util.concurrent.atomic.AtomicLong import kafka.common.TopicAndPartition -import org.apache.kafka.clients.consumer.{ConsumerRecord, ConsumerConfig} +import org.apache.kafka.clients.consumer.{ ConsumerRecord, ConsumerConfig } import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.scheduler.rate.RateEstimator -import org.apache.spark.streaming.scheduler.{StreamingListenerBatchCompleted, StreamingListenerBatchStarted, StreamingListenerBatchSubmitted, StreamingListener} -import org.apache.spark.streaming.{Time, Milliseconds, StreamingContext} +import org.apache.spark.streaming.scheduler.{ StreamingListenerBatchCompleted, StreamingListenerBatchStarted, StreamingListenerBatchSubmitted, StreamingListener } +import org.apache.spark.streaming.{ Time, Milliseconds, StreamingContext } import org.apache.spark.util.Utils -import org.apache.spark.{SparkContext, SparkConf, Logging, SparkFunSuite} +import org.apache.spark.{ SparkContext, SparkConf, Logging, SparkFunSuite } import org.scalatest.concurrent.Eventually -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.{ BeforeAndAfter, BeforeAndAfterAll } import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -78,7 +78,6 @@ class DirectKafkaStreamSuite } } - test("basic stream receiving with multiple topics and earliest starting offset") { val topics = Set("new_basic1", "new_basic2", "new_basic3") val data = Map("a" -> 7, "b" -> 9) @@ -94,8 +93,7 @@ class DirectKafkaStreamSuite "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "1000" - ) + "spark.kafka.poll.time" -> "1000") ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { @@ -129,8 +127,9 @@ class DirectKafkaStreamSuite // 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") + collected.foreach { + case (partSize, rangeSize) => + assert(partSize === rangeSize, "offset ranges are wrong") } } stream.foreachRDD { rdd => allReceived ++= rdd.collect() } @@ -154,8 +153,7 @@ class DirectKafkaStreamSuite "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "100" - ) + "spark.kafka.poll.time" -> "100") val kc = new KafkaCluster(kafkaParams) def getLatestOffset(): Long = { kc.getLatestOffsets(Set(topicPartition)).right.get.get(topicPartition).get @@ -177,8 +175,7 @@ class DirectKafkaStreamSuite assert( stream.asInstanceOf[DirectKafkaInputDStream[_, _, _]] .fromOffsets(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) + "Start offset not from latest") val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] stream.map { @@ -193,7 +190,6 @@ class DirectKafkaStreamSuite assert(!collectedData.contains("a")) } - test("creating stream by offset") { val topic = "new_offset" val topicPartition = TopicAndPartition(topic, 0) @@ -206,8 +202,7 @@ class DirectKafkaStreamSuite "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "100" - ) + "spark.kafka.poll.time" -> "100") val kc = new KafkaCluster(kafkaParams) def getLatestOffset(): Long = { kc.getLatestOffsets(Set(topicPartition)).right.get.get(topicPartition).get @@ -230,8 +225,7 @@ class DirectKafkaStreamSuite assert( stream.asInstanceOf[DirectKafkaInputDStream[_, _, _]] .fromOffsets(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) + "Start offset not from latest") val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] stream.foreachRDD { rdd => collectedData ++= rdd.collect() } @@ -257,8 +251,7 @@ class DirectKafkaStreamSuite "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "1000" - ) + "spark.kafka.poll.time" -> "1000") // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { @@ -313,8 +306,7 @@ class DirectKafkaStreamSuite offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, - "starting offset not zero" - ) + "starting offset not zero") ssc.stop() logInfo("====== RESTARTING ========") @@ -330,8 +322,7 @@ class DirectKafkaStreamSuite recoveredOffsetRanges.forall { or => earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) }, - "Recovered ranges are not the same as the ones generated" - ) + "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() @@ -356,8 +347,7 @@ class DirectKafkaStreamSuite "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "1000" - ) + "spark.kafka.poll.time" -> "1000") import DirectKafkaStreamSuite._ ssc = new StreamingContext(sparkConf, Milliseconds(200)) @@ -397,8 +387,7 @@ class DirectKafkaStreamSuite "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "1000" - ) + "spark.kafka.poll.time" -> "1000") val batchIntervalMilliseconds = 100 val estimator = new ConstantEstimator(100) @@ -449,8 +438,8 @@ class DirectKafkaStreamSuite Seq(100, 50, 20).foreach { rate => collectedData.clear() // Empty this buffer on each pass. estimator.updateRate(rate) // Set a new rate. - // Expect blocks of data equal to "rate", scaled by the interval length in secs. - val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) + // Expect blocks of data equal to "rate", scaled by the interval length in secs. + val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) kafkaTestUtils.sendMessages(topic, messages) eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { // Assert that rate estimator values are used to determine maxMessagesPerPartition. @@ -464,8 +453,7 @@ class DirectKafkaStreamSuite } /** Get the generated offset ranges from the DirectKafkaStream */ - private def getOffsetRanges[K, V](kafkaStream: DStream[(K, V)]): - Seq[(Time, Array[OffsetRange])] = { + 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 { @@ -506,10 +494,9 @@ private[streaming] class ConstantEstimator(@volatile private var rate: Long) } def compute( - time: Long, - elements: Long, - processingDelay: Long, - schedulingDelay: Long): Option[Double] = Some(rate) + time: Long, + elements: Long, + processingDelay: Long, + schedulingDelay: Long): Option[Double] = Some(rate) } - diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala index 980ad65fbe8c..e6ce84e3a905 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.kafka.v09 import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.spark.SparkFunSuite -import org.apache.spark.streaming.kafka.v09.KafkaTestUtils import org.scalatest.BeforeAndAfterAll import scala.util.Random @@ -43,8 +42,7 @@ class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> - "org.apache.kafka.common.serialization.StringDeserializer" - ) + "org.apache.kafka.common.serialization.StringDeserializer") newKc = new KafkaCluster(kafkaParams) } @@ -63,5 +61,4 @@ class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { assert(latest(topicPartition) === 1, "didn't get latest") } - } diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala index 5612a5876de2..1c2fd5f0a61d 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala @@ -18,9 +18,8 @@ package org.apache.spark.streaming.kafka.v09 import kafka.common.TopicAndPartition -import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} +import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } import org.apache.spark._ -import org.apache.spark.streaming.kafka.v09.{HasOffsetRanges, KafkaTestUtils, KafkaUtils, OffsetRange} import org.scalatest.BeforeAndAfterAll import scala.util.Random @@ -65,8 +64,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "10000" - ) + "spark.kafka.poll.time" -> "10000") val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) @@ -97,7 +95,6 @@ class KafkaRDDSuite extends SparkFunSuite 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 = s"new_topicboundary-${Random.nextInt}" @@ -112,8 +109,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "1000" - ) + "spark.kafka.poll.time" -> "1000") val kc = new KafkaCluster(kafkaParams) @@ -164,14 +160,14 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { kc.getCommittedOffsets(topicPartitions).right.toOption.orElse( kc.getEarliestOffsets(topicPartitions).right.toOption.map { offs => offs.map(kv => kv._1 -> kv._2) - } - ) + }) } kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => consumerOffsets(topicPartitions).flatMap { from => kc.getLatestOffsets(topicPartitions).right.toOption.map { until => - val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) => - OffsetRange(tp.topic, tp.partition, fromOffset, until(tp)) + val offsetRanges = from.map { + case (tp: TopicAndPartition, fromOffset: Long) => + OffsetRange(tp.topic, tp.partition, fromOffset, until(tp)) }.toArray KafkaUtils.createRDD[String, String, String]( diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala index 558f57372c10..17842f5ff77b 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala @@ -19,9 +19,8 @@ package org.apache.spark.streaming.kafka.v09 import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.kafka.v09.{KafkaTestUtils, KafkaUtils} -import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.streaming.{ Milliseconds, StreamingContext } +import org.apache.spark.{ SparkConf, SparkFunSuite } import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually @@ -67,8 +66,7 @@ class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfter "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "100" - ) + "spark.kafka.poll.time" -> "100") val stream = KafkaUtils.createStream[String, String]( ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala index 139f8573a0e7..f7d55f722603 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala @@ -23,11 +23,11 @@ import kafka.common.TopicAndPartition import kafka.utils.ZkUtils import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext} +import org.apache.spark.streaming.{ Milliseconds, StreamingContext } import org.apache.spark.util.Utils -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{ SparkConf, SparkFunSuite } import org.scalatest.concurrent.Eventually -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.{ BeforeAndAfter, BeforeAndAfterAll } import scala.collection.mutable import scala.concurrent.duration._ @@ -35,7 +35,7 @@ import scala.language.postfixOps import scala.util.Random class ReliableKafkaStreamSuite extends SparkFunSuite -with BeforeAndAfterAll with BeforeAndAfter with Eventually { + with BeforeAndAfterAll with BeforeAndAfter with Eventually { private val sparkConf = new SparkConf() .setMaster("local[4]") @@ -53,7 +53,7 @@ with BeforeAndAfterAll with BeforeAndAfter with Eventually { private var kc: KafkaCluster[_, _] = null - override def beforeAll() : Unit = { + override def beforeAll(): Unit = { kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() @@ -67,13 +67,12 @@ with BeforeAndAfterAll with BeforeAndAfter with Eventually { "org.apache.kafka.common.serialization.StringDeserializer", ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "100" - ) + "spark.kafka.poll.time" -> "100") kc = new KafkaCluster(kafkaParams) tempDirectory = Utils.createTempDir() - zkUtils = ZkUtils(kafkaTestUtils.zookeeperClient, false) + zkUtils = ZkUtils(kafkaTestUtils.zookeeperClient, false) } override def afterAll(): Unit = { @@ -130,9 +129,10 @@ with BeforeAndAfterAll with BeforeAndAfter with Eventually { test("Reliable Kafka input stream with multiple topics") { val topics = Map("new_topic1" -> 1, "new_topic2" -> 1, "new_topic3" -> 1) - topics.foreach { case (t, _) => - kafkaTestUtils.createTopic(t) - kafkaTestUtils.sendMessages(t, data) + topics.foreach { + case (t, _) => + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) } // Before started, verify all the group/topic/partition offsets are 0. @@ -150,7 +150,6 @@ with BeforeAndAfterAll with BeforeAndAfter with Eventually { } } - /** Getting partition offset from Zookeeper. */ private def getCommitOffset(topic: String, partition: Int): Option[Long] = { kc.getCommittedOffset(TopicAndPartition(topic, partition)).right.toOption From 2379834aa92b127f1635013bbe3046e37464cbed Mon Sep 17 00:00:00 2001 From: nikit-os Date: Wed, 23 Dec 2015 14:22:52 +0200 Subject: [PATCH 03/16] Add SSL configuration for Kafka --- external/kafka-v09/pom.xml | 4 + .../streaming/kafka/v09/KafkaUtils.scala | 58 ++++++++++--- .../kafka/v09/KafkaUtilsSSLSuite.scala | 87 +++++++++++++++++++ 3 files changed, 136 insertions(+), 13 deletions(-) create mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaUtilsSSLSuite.scala diff --git a/external/kafka-v09/pom.xml b/external/kafka-v09/pom.xml index e30bd9289464..8392b12459af 100644 --- a/external/kafka-v09/pom.xml +++ b/external/kafka-v09/pom.xml @@ -48,6 +48,10 @@ 2.2.0 + + org.eclipse.jetty + jetty-util + org.apache.spark diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala index 67e6690d93ef..c6f09cf1c47c 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala @@ -19,19 +19,21 @@ package org.apache.spark.streaming.kafka.v09 import java.io.OutputStream import java.lang.{Integer => JInt, Long => JLong} -import java.util.{List => JList, Map => JMap, Set => JSet} +import java.util.{Map => JMap, Set => JSet} + +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.SslConfigs import scala.reflect.ClassTag import com.google.common.base.Charsets.UTF_8 import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.{DefaultDecoder, Decoder, StringDecoder} +import kafka.serializer.Decoder import net.razorvine.pickle.{Opcodes, Pickler, IObjectPickler} import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.streaming.util.WriteAheadLogUtils -import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.{SSLOptions, SparkContext, SparkException} import scala.collection.JavaConverters._ import scala.reflect._ @@ -41,10 +43,35 @@ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java._ -import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} +import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} object KafkaUtils { + def addSSLOptions( + kafkaParams: Map[String, String], + sc: SparkContext + ): Map[String, String] = { + + val sparkConf = sc.getConf + val defaultSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl", None) + val kafkaSSLOptions = SSLOptions.parse(sparkConf, "spark.ssl.kafka", Some(defaultSSLOptions)) + + if (kafkaSSLOptions.enabled) { + val sslParams = Map[String, Option[_]]( + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG -> Some("SSL"), + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG -> kafkaSSLOptions.trustStore, + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG -> kafkaSSLOptions.trustStorePassword, + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG -> kafkaSSLOptions.keyStore, + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG -> kafkaSSLOptions.keyStorePassword, + SslConfigs.SSL_KEY_PASSWORD_CONFIG -> kafkaSSLOptions.keyPassword + ) + kafkaParams ++ sslParams.filter(_._2.isDefined).mapValues(_.get.toString) + } else { + kafkaParams + } + + } + /** * Create an input stream that pulls messages from Kafka Brokers. * @param ssc StreamingContext object @@ -91,7 +118,12 @@ object KafkaUtils { storageLevel: StorageLevel ): ReceiverInputDStream[(K, V)] = { val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) - new KafkaInputDStream[K, V](ssc, kafkaParams, topics, walEnabled, storageLevel) + new KafkaInputDStream[K, V]( + ssc, + addSSLOptions(kafkaParams, ssc.sparkContext), + topics, + walEnabled, + storageLevel) } /** @@ -149,9 +181,9 @@ object KafkaUtils { offsetRanges: Array[OffsetRange] ): RDD[(K, V)] = sc.withScope { val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) - val kc = new KafkaCluster[K, V](kafkaParams) + val kc = new KafkaCluster[K, V](addSSLOptions(kafkaParams, sc)) checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, (K, V)](sc, kafkaParams, offsetRanges, messageHandler) + new KafkaRDD[K, V, (K, V)](sc, addSSLOptions(kafkaParams, sc), offsetRanges, messageHandler) } /** @@ -174,10 +206,10 @@ object KafkaUtils { offsetRanges: Array[OffsetRange], messageHandler: ConsumerRecord[K, V] => R ): RDD[R] = sc.withScope { - val kc = new KafkaCluster[K, V](kafkaParams) + val kc = new KafkaCluster[K, V](addSSLOptions(kafkaParams, sc)) val cleanedHandler = sc.clean(messageHandler) checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, R](sc, kafkaParams, offsetRanges, cleanedHandler) + new KafkaRDD[K, V, R](sc, addSSLOptions(kafkaParams, sc), offsetRanges, cleanedHandler) } /** @@ -269,7 +301,7 @@ object KafkaUtils { ): InputDStream[R] = { val cleanedHandler = ssc.sc.clean(messageHandler) new DirectKafkaInputDStream[K, V, R]( - ssc, kafkaParams, fromOffsets, messageHandler) + ssc, addSSLOptions(kafkaParams, ssc.sparkContext), fromOffsets, messageHandler) } /** @@ -308,7 +340,7 @@ object KafkaUtils { topics: Set[String] ): InputDStream[(K, V)] = { val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) - val kc = new KafkaCluster[K, V](kafkaParams) + val kc = new KafkaCluster[K, V](addSSLOptions(kafkaParams, ssc.sparkContext)) val reset = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).map(_.toLowerCase) val fromOffsets = if (reset == Some("earliest")) { @@ -320,7 +352,7 @@ object KafkaUtils { kc.close() new DirectKafkaInputDStream[K, V, (K, V)]( - ssc, kafkaParams, fromOffsets, messageHandler) + ssc, addSSLOptions(kafkaParams, ssc.sparkContext), fromOffsets, messageHandler) } /** diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaUtilsSSLSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaUtilsSSLSuite.scala new file mode 100644 index 000000000000..4ffb7bd53c90 --- /dev/null +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaUtilsSSLSuite.scala @@ -0,0 +1,87 @@ +/* + * 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.v09 + +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.config.SslConfigs +import org.apache.spark.{SparkContext, SparkConf, SparkFunSuite} +import org.scalatest.BeforeAndAfterAll + +class KafkaUtilsSSLSuite extends SparkFunSuite with BeforeAndAfterAll { + private var sc: SparkContext = _ + + private val pathToKeyStore = "/path/to/ssl_keystore" + private val pathToTrustStore = "/path/to/ssl_truststore" + private val keystorePasswd = "keystore_secret_pass" + private val truststorePasswd = "truststore_secret_pass" + private val keyPasswd = "key_secret_pass" + + private val sparkSslProperties = Map[String, String] ( + "spark.ssl.kafka.enabled" -> "true", + "spark.ssl.kafka.keyStore" -> pathToKeyStore, + "spark.ssl.kafka.keyStorePassword" -> keystorePasswd, + "spark.ssl.kafka.trustStore" -> pathToTrustStore, + "spark.ssl.kafka.trustStorePassword" -> truststorePasswd, + "spark.ssl.kafka.keyPassword" -> keyPasswd + ) + + private val kafkaSslProperties = Map[String, String] ( + CommonClientConfigs.SECURITY_PROTOCOL_CONFIG -> "SSL", + SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG -> pathToKeyStore, + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG -> keystorePasswd, + SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG -> pathToTrustStore, + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG -> truststorePasswd, + SslConfigs.SSL_KEY_PASSWORD_CONFIG -> keyPasswd + ) + + val sparkConf = new SparkConf() + .setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + + override def beforeAll { + sparkConf.setAll(sparkSslProperties) + sc = new SparkContext(sparkConf) + } + + override def afterAll { + if (sc != null) { + sc.stop + sc = null + } + } + + test("Check adding SSL properties to Kafka parameters") { + val kafkaParams = Map( + ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> "localhost:9093", + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", + ConsumerConfig.GROUP_ID_CONFIG -> "test-consumer", + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> + "org.apache.kafka.common.serialization.StringDeserializer", + "spark.kafka.poll.time" -> "100") + + val kafkaParamsWithSSL = KafkaUtils.addSSLOptions(kafkaParams, sc) + + kafkaSslProperties.foreach { + case (k, v) => assert(kafkaParamsWithSSL.get(k).get.toString == v) + } + } + +} From 2db272fe025650e869032bebd8e035cb06144501 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Fri, 25 Dec 2015 17:42:55 +0200 Subject: [PATCH 04/16] Remove kafkaConsumer.wakeup() from ReliableReceiver --- .../apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala index dbaa62dd4f9d..da7f08827cd1 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala @@ -239,7 +239,6 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( val offsetAndMetadata = new OffsetAndMetadata(offset) offsets.put(topicPartition, offsetAndMetadata) val lock = consumerAndLockMap(kafkaConsumer) - kafkaConsumer.wakeup() lock.lock() try { kafkaConsumer.commitSync(offsets) From d848519cc1840af8ab9d6e3f68a1431c78139b30 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Mon, 28 Dec 2015 13:20:30 +0200 Subject: [PATCH 05/16] Enable fair ordering policy for ReentrantLock --- .../spark/streaming/kafka/v09/ReliableKafkaReceiver.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala index da7f08827cd1..bae8d717ccc8 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala @@ -135,7 +135,7 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( topicAndPartition, newConsumer ) - consumerAndLockMap.put(newConsumer, new ReentrantLock()) + consumerAndLockMap.put(newConsumer, new ReentrantLock(true)) newConsumer.subscribe(Collections.singletonList[String](topicAndPartition.topic)) messageHandlerThreadPool.submit(new MessageHandler(newConsumer)) } From 44bb56806ab5fb86d0e27ed8108188c5808bae0f Mon Sep 17 00:00:00 2001 From: nikit-os Date: Tue, 29 Dec 2015 16:56:17 +0200 Subject: [PATCH 06/16] KafkaCluster fixes --- .../kafka/v09/DirectKafkaInputDStream.scala | 2 +- .../streaming/kafka/v09/KafkaCluster.scala | 150 +++++------------- .../kafka/v09/KafkaInputDStream.scala | 4 +- .../streaming/kafka/v09/KafkaUtils.scala | 18 +-- .../kafka/v09/ReliableKafkaReceiver.scala | 4 +- .../kafka/v09/DirectKafkaStreamSuite.scala | 9 +- .../kafka/v09/KafkaClusterSuite.scala | 4 +- .../streaming/kafka/v09/KafkaRDDSuite.scala | 34 ++-- .../kafka/v09/ReliableKafkaStreamSuite.scala | 10 +- 9 files changed, 86 insertions(+), 149 deletions(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala index dac33a097247..bb7e179f41d0 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala @@ -109,7 +109,7 @@ class DirectKafkaInputDStream[ protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = { val topicPartition = fromOffsets.map(fo => fo._1).toSet - kafkaCluster.getLatestOffsets(topicPartition).right.get + kafkaCluster.getLatestOffsets(topicPartition) } // limits the maximum number of messages per partition diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala index c71bacbfbde8..fc3d12be901f 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala @@ -18,16 +18,14 @@ package org.apache.spark.streaming.kafka.v09 import java.util -import java.util.{Properties, Collections} +import java.util.{Collections} import kafka.common.TopicAndPartition -import org.apache.kafka.clients.consumer.{KafkaConsumer, OffsetAndMetadata} +import org.apache.kafka.clients.consumer.{OffsetResetStrategy, KafkaConsumer, OffsetAndMetadata} import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkException -import org.apache.spark.streaming.kafka.v09.KafkaCluster.SeekType.SeekType import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer import scala.reflect._ import scala.util.control.NonFatal @@ -41,45 +39,37 @@ private[spark] class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String]) extends Serializable { - import KafkaCluster.{Err, SeekType, toTopicPart} + import KafkaCluster.{toTopicPart} - def getLatestOffsets(topicPartitions: Set[TopicAndPartition]): - Either[Err, Map[TopicAndPartition, Long]] = { - seek(topicPartitions, SeekType.End) + def getLatestOffsets(topicPartitions: Set[TopicAndPartition]): Map[TopicAndPartition, Long] = { + seek(topicPartitions, OffsetResetStrategy.LATEST) } - def getEarliestOffsets(topicPartitions: Set[TopicAndPartition]): - Either[Err, Map[TopicAndPartition, Long]] = { - seek(topicPartitions, SeekType.Beginning) + def getEarliestOffsets(topicPartitions: Set[TopicAndPartition]): Map[TopicAndPartition, Long] = { + seek(topicPartitions, OffsetResetStrategy.EARLIEST) } - def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = { - val errs = new Err - var result: Either[Err, Set[TopicAndPartition]] = null - - withConsumer(errs)(consumer => { - try { - val partInfo = topics.flatMap(topic => consumer.partitionsFor(topic).asScala) + def getPartitions(topics: Set[String]): Set[TopicAndPartition] = { + withConsumer { consumer => { + val partInfo = topics.flatMap { + topic => Option(consumer.partitionsFor(topic)) match { + case None => throw new SparkException("Topic doesn't exist " + topic) + case Some(partInfoList) => partInfoList.asScala.toList + } + } val topicPartitions: Set[TopicAndPartition] = partInfo.map { partition => new TopicAndPartition(partition.topic(), partition.partition()) } - result = Right(topicPartitions) - } catch { - case NonFatal(e) => { - errs.append(e) - result = Left(errs) - } + topicPartitions } - }) - result + }.asInstanceOf[Set[TopicAndPartition]] } def setConsumerOffsets(offsets: Map[TopicAndPartition, Long]): Unit = { - val errs = new Err val topicPartOffsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() val topicAndPartition = offsets.map(tpl => tpl._1).toSeq - withConsumer(errs)(consumer => { + withConsumer(consumer => { consumer.assign(Collections.emptyList[TopicPartition]) consumer.assign(topicAndPartition.map(tp => toTopicPart(tp)).asJava) @@ -93,79 +83,42 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String }) } - def getCommittedOffset(topicAndPartition: TopicAndPartition): Either[Err, Long] = { - val errs = new Err - var result: Either[Err, Long] = null - - withConsumer(errs)(consumer => { - try { - val topicPartition = toTopicPart(topicAndPartition) - consumer.assign(util.Arrays.asList(topicPartition)) - result = Right(consumer.committed(topicPartition).offset()) - } catch { - case NonFatal(e) => { - errs.append(e) - result = Left(errs) - } - } - }) - result - } - def getCommittedOffsets(topicAndPartitions: Set[TopicAndPartition]): - Either[Err, Map[TopicAndPartition, Long]] = { - val errs = new Err - var result: Either[Err, Map[TopicAndPartition, Long]] = null - - withConsumer(errs)(consumer => { - try { - val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) - consumer.assign(topicPartitions.toList.asJava) - result = Right(topicAndPartitions.map( - tp => tp -> (consumer.committed(toTopicPart(tp))).offset()).toMap) - } catch { - case NonFatal(e) => { - errs.append(e) - result = Left(errs) + Map[TopicAndPartition, Long] = { + withConsumer(consumer => { + val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) + consumer.assign(topicPartitions.toList.asJava) + topicAndPartitions.map( tp => { + val offsetAndMetadata = consumer.committed(toTopicPart(tp)) + Option(offsetAndMetadata) match { + case None => throw new SparkException(s"Topic $tp hasn't committed offsets") + case Some(om) => tp -> om.offset() } } - }) - result + ).toMap + }).asInstanceOf[Map[TopicAndPartition, Long]] } - def seek(topicAndPartitions: Set[TopicAndPartition], seekType: SeekType): - Either[Err, Map[TopicAndPartition, Long]] = { - val errs = new Err - var result: Either[Err, Map[TopicAndPartition, Long]] = null - - withConsumer(errs)(consumer => { - try { - val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) - consumer.assign(topicPartitions.toList.asJava) - seekType match { - case SeekType.Beginning => consumer.seekToBeginning(topicPartitions.toArray: _*) - case SeekType.End => consumer.seekToEnd(topicPartitions.toArray: _*) - } - result = Right(topicAndPartitions.map( - tp => tp -> (consumer.position(toTopicPart(tp))) - ).toMap) - } catch { - case NonFatal(e) => { - errs.append(e) - result = Left(errs) - } + def seek(topicAndPartitions: Set[TopicAndPartition], resetStrategy: OffsetResetStrategy): + Map[TopicAndPartition, Long] = { + withConsumer(consumer => { + val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) + consumer.assign(topicPartitions.toList.asJava) + resetStrategy match { + case OffsetResetStrategy.EARLIEST => consumer.seekToBeginning(topicPartitions.toArray: _*) + case OffsetResetStrategy.LATEST => consumer.seekToEnd(topicPartitions.toArray: _*) + case _ => throw new SparkException("Unknown OffsetResetStrategy " + resetStrategy) } - }) - result - + topicAndPartitions.map( + tp => tp -> (consumer.position(toTopicPart(tp))) + ).toMap + }).asInstanceOf[Map[TopicAndPartition, Long]] } - private def withConsumer(errs: Err)(fn: KafkaConsumer[K, V] => Any): Unit = { + private def withConsumer(fn: KafkaConsumer[K, V] => Any): Any = { var consumer: KafkaConsumer[K, V] = null try { - val props = new Properties() - kafkaParams.foreach(param => props.put(param._1, param._2)) - consumer = new KafkaConsumer[K, V](props) + consumer = new KafkaConsumer[K, V](kafkaParams.asInstanceOf[Map[String, Object]].asJava) fn(consumer) } finally { if (consumer != null) { @@ -181,21 +134,6 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String object KafkaCluster { - object SeekType extends Enumeration { - type SeekType = Value - val Beginning, End = Value - } - - type Err = ArrayBuffer[Throwable] - - /** If the result is right, return it, otherwise throw SparkException */ - def checkErrors[T](result: Either[Err, T]): T = { - result.fold( - errs => throw new SparkException(errs.mkString("\n")), - ok => ok - ) - } - def toTopicPart(topicAndPartition: TopicAndPartition): TopicPartition = { new TopicPartition(topicAndPartition.topic, topicAndPartition.partition) } diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala index 30f10527cabe..0274e86db683 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala @@ -92,8 +92,8 @@ V: ClassTag]( "KafkaMessageHandler") try { // Start the messages handler for each partition - val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet).right.toOption - val iter = topicAndPartitions.get.iterator + val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet) + val iter = topicAndPartitions.iterator while (iter.hasNext) { val topicAndPartition = iter.next() diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala index c6f09cf1c47c..9ae8afcc5eac 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala @@ -159,16 +159,12 @@ object KafkaUtils { kc: KafkaCluster[_, _], offsetRanges: Array[OffsetRange]): Unit = { val topics = offsetRanges.map(_.topicAndPartition).toSet - val result = for { - low <- kc.getEarliestOffsets(topics).right - high <- kc.getLatestOffsets(topics).right - } yield { - offsetRanges.filterNot { o => - low(o.topicAndPartition) <= o.fromOffset && - o.untilOffset <= high(o.topicAndPartition) - } + val low = kc.getEarliestOffsets(topics) + val high = kc.getLatestOffsets(topics) + val badRanges = offsetRanges.filterNot { o => + low(o.topicAndPartition()) <= o.fromOffset && + o.untilOffset <= high(o.topicAndPartition()) } - val badRanges = KafkaCluster.checkErrors(result) if (!badRanges.isEmpty) { throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) } @@ -344,9 +340,9 @@ object KafkaUtils { val reset = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).map(_.toLowerCase) val fromOffsets = if (reset == Some("earliest")) { - kc.getEarliestOffsets(kc.getPartitions(topics).right.get).right.get + kc.getEarliestOffsets(kc.getPartitions(topics)) } else { - kc.getLatestOffsets(kc.getPartitions(topics).right.get).right.get + kc.getLatestOffsets(kc.getPartitions(topics)) } kc.close() diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala index bae8d717ccc8..fcec93244f8b 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala @@ -126,8 +126,8 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( try { // Start the messages handler for each partition - val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet).right.toOption - val iter = topicAndPartitions.get.iterator + val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet) + val iter = topicAndPartitions.iterator while (iter.hasNext) { val topicAndPartition = iter.next() val newConsumer = new KafkaConsumer[K, V](props) diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala index f07d635850a4..613400432589 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala @@ -156,7 +156,7 @@ class DirectKafkaStreamSuite "spark.kafka.poll.time" -> "100") val kc = new KafkaCluster(kafkaParams) def getLatestOffset(): Long = { - kc.getLatestOffsets(Set(topicPartition)).right.get.get(topicPartition).get + kc.getLatestOffsets(Set(topicPartition)).get(topicPartition).get } // Send some initial messages before starting context @@ -205,7 +205,7 @@ class DirectKafkaStreamSuite "spark.kafka.poll.time" -> "100") val kc = new KafkaCluster(kafkaParams) def getLatestOffset(): Long = { - kc.getLatestOffsets(Set(topicPartition)).right.get.get(topicPartition).get + kc.getLatestOffsets(Set(topicPartition)).get(topicPartition).get } // Send some initial messages before starting context @@ -407,7 +407,7 @@ class DirectKafkaStreamSuite val kafkaStream = withClue("Error creating direct stream") { val kc = new KafkaCluster(kafkaParams) val messageHandler = (mmd: ConsumerRecord[String, String]) => (mmd.key(), mmd.value()) - val m = kc.getEarliestOffsets(Set(topicPartition)).right.get + val m = kc.getEarliestOffsets(Set(topicPartition)) new DirectKafkaInputDStream[String, String, (String, String)]( ssc, kafkaParams, m, messageHandler) { @@ -453,7 +453,8 @@ class DirectKafkaStreamSuite } /** Get the generated offset ranges from the DirectKafkaStream */ - private def getOffsetRanges[K, V](kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { + 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 { diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala index e6ce84e3a905..93293acd2b49 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala @@ -54,10 +54,10 @@ class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { } test("leader offset apis") { - val earliest = newKc.getEarliestOffsets(Set(topicPartition)).right.get + val earliest = newKc.getEarliestOffsets(Set(topicPartition)) assert(earliest(topicPartition) === 0, "didn't get earliest") - val latest = newKc.getLatestOffsets(Set(topicPartition)).right.get + val latest = newKc.getLatestOffsets(Set(topicPartition)) assert(latest(topicPartition) === 1, "didn't get latest") } diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala index 1c2fd5f0a61d..8b00e6a759f2 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala @@ -156,25 +156,21 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { // get an rdd from the committed consumer offsets until the latest leader offsets, private def getRdd(kc: KafkaCluster[_, _], topics: Set[String]) = { val groupId = kc.kafkaParams("group.id") - def consumerOffsets(topicPartitions: Set[TopicAndPartition]) = { - kc.getCommittedOffsets(topicPartitions).right.toOption.orElse( - kc.getEarliestOffsets(topicPartitions).right.toOption.map { offs => - offs.map(kv => kv._1 -> kv._2) - }) + val topicPartitions = kc.getPartitions(topics) + val consumerOffsets = try { + kc.getCommittedOffsets(topicPartitions) + } catch { + case e: SparkException => kc.getEarliestOffsets(topicPartitions) } - kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => - consumerOffsets(topicPartitions).flatMap { from => - kc.getLatestOffsets(topicPartitions).right.toOption.map { until => - val offsetRanges = from.map { - case (tp: TopicAndPartition, fromOffset: Long) => - OffsetRange(tp.topic, tp.partition, fromOffset, until(tp)) - }.toArray - - KafkaUtils.createRDD[String, String, String]( - sc, kc.kafkaParams, offsetRanges, - (cr: ConsumerRecord[String, String]) => s"${cr.offset()} ${cr.value()}") - } - } + val latestOffsets = kc.getLatestOffsets(topicPartitions) + + val offsetRanges = consumerOffsets.map { case (tp: TopicAndPartition, fromOffset: Long) => + OffsetRange(tp.topic, tp.partition, fromOffset, latestOffsets(tp)) + }.toArray + + Option(KafkaUtils.createRDD[String, String, String]( + sc, kc.kafkaParams, offsetRanges, + (cr: ConsumerRecord[String, String]) => s"${cr.offset()} ${cr.value()}")) } - } + } diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala index f7d55f722603..235d10869c5d 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala @@ -25,7 +25,7 @@ import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{ Milliseconds, StreamingContext } import org.apache.spark.util.Utils -import org.apache.spark.{ SparkConf, SparkFunSuite } +import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} import org.scalatest.concurrent.Eventually import org.scalatest.{ BeforeAndAfter, BeforeAndAfterAll } @@ -152,6 +152,12 @@ class ReliableKafkaStreamSuite extends SparkFunSuite /** Getting partition offset from Zookeeper. */ private def getCommitOffset(topic: String, partition: Int): Option[Long] = { - kc.getCommittedOffset(TopicAndPartition(topic, partition)).right.toOption + try { + val tp = TopicAndPartition(topic, partition) + Option(kc.getCommittedOffsets(Set(tp))(tp)) + } catch { + case e: SparkException => None + } + } } From 85fad9410ab5133bd92e01af278e2f473c07b745 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Tue, 29 Dec 2015 17:30:20 +0200 Subject: [PATCH 07/16] Add missing assert --- .../scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala index 5a454218dd83..3b9df97fe082 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala @@ -158,6 +158,7 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( } if (!iter.hasNext) { + assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) finished = true null.asInstanceOf[R] } else { From 2d36c76f1bc8bcba59261d37bbff90e67ece73d7 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Tue, 29 Dec 2015 17:27:46 +0200 Subject: [PATCH 08/16] Bring back getPreferredLocations() in KafkaRDD --- .../streaming/kafka/v09/KafkaCluster.scala | 20 ++++++++++++++++++- .../spark/streaming/kafka/v09/KafkaRDD.scala | 13 +++++++++++- .../kafka/v09/KafkaRDDPartition.scala | 3 ++- .../kafka/v09/DirectKafkaStreamSuite.scala | 2 +- 4 files changed, 34 insertions(+), 4 deletions(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala index fc3d12be901f..5d1885f61a69 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala @@ -22,7 +22,7 @@ import java.util.{Collections} import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.{OffsetResetStrategy, KafkaConsumer, OffsetAndMetadata} -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{PartitionInfo, TopicPartition} import org.apache.spark.SparkException import scala.collection.JavaConverters._ @@ -65,6 +65,24 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String }.asInstanceOf[Set[TopicAndPartition]] } + def getPartitionsLeader(topics: Set[String]): Map[TopicPartition, String] = { + getPartitionInfo(topics).map { pi => + new TopicPartition(pi.topic, pi.partition) -> pi.leader.host + }.toMap + } + + def getPartitionInfo(topics: Set[String]): Set[PartitionInfo] = { + withConsumer { consumer => + topics.flatMap { topic => + Option(consumer.partitionsFor(topic)) match { + case None => + throw new SparkException("Topic doesn't exist " + topic) + case Some(piList) => piList.asScala.toList + } + } + }.asInstanceOf[Set[PartitionInfo]] + } + def setConsumerOffsets(offsets: Map[TopicAndPartition, Long]): Unit = { val topicPartOffsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() val topicAndPartition = offsets.map(tpl => tpl._1).toSeq diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala index 3b9df97fe082..5e08c8d2c02f 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala @@ -24,6 +24,7 @@ import org.apache.kafka.clients.consumer.{ ConsumerRecord, KafkaConsumer } import org.apache.kafka.common.TopicPartition import org.apache.spark.partial.{ BoundedDouble, PartialResult } import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.kafka.v09.KafkaCluster import org.apache.spark.streaming.kafka.v09.KafkaCluster.toTopicPart import org.apache.spark.util.NextIterator import org.apache.spark.{ Logging, Partition, SparkContext, TaskContext } @@ -53,9 +54,13 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt override def getPartitions: Array[Partition] = { + val topics = offsetRanges.map { _.topic }.toSet + val tpLeaders = new KafkaCluster[K, V](kafkaParams).getPartitionsLeader(topics) + offsetRanges.zipWithIndex.map { case (o, i) => - new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, + tpLeaders(new TopicPartition(o.topic, o.partition))) }.toArray } @@ -99,6 +104,12 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( buf.toArray } + override def getPreferredLocations(thePart: Partition): Seq[String] = { + val part = thePart.asInstanceOf[KafkaRDDPartition] + // TODO is additional hostname resolution necessary here + Seq(part.host) + } + private def errBeginAfterEnd(part: KafkaRDDPartition): String = s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + s"for topic ${part.topic} partition ${part.partition}. " + diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala index 0df2a8a3fd8f..b3889aae9063 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDPartition.scala @@ -30,7 +30,8 @@ class KafkaRDDPartition( val topic: String, val partition: Int, val fromOffset: Long, - val untilOffset: Long + val untilOffset: Long, + val host: String ) extends Partition { /** Number of messages this partition refers to */ def count(): Long = untilOffset - fromOffset diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala index 613400432589..7f6697a1382a 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala @@ -261,7 +261,7 @@ class DirectKafkaStreamSuite kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1 }.toMap) - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(20 seconds), interval(50 milliseconds)) { assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) From ca4f4d4b08b7a18cb6371b43a767aa3347f905dc Mon Sep 17 00:00:00 2001 From: nikit-os Date: Mon, 4 Jan 2016 18:46:47 +0200 Subject: [PATCH 09/16] Minor fixes in JavaTestSuites --- .../kafka/v09/JavaDirectKafkaStreamSuite.java | 37 +++++------ .../kafka/v09/JavaKafkaRDDSuite.java | 33 +++------- .../kafka/v09/JavaKafkaStreamSuite.java | 61 +++++++++---------- 3 files changed, 54 insertions(+), 77 deletions(-) diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java index 17b5b1a8607a..e9ecc279230a 100644 --- a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java @@ -19,11 +19,8 @@ import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.common.TopicPartition; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -31,10 +28,6 @@ import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.api.java.JavaDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; -import org.apache.spark.streaming.kafka.v09.HasOffsetRanges; -import org.apache.spark.streaming.kafka.v09.KafkaTestUtils; -import org.apache.spark.streaming.kafka.v09.KafkaUtils; -import org.apache.spark.streaming.kafka.v09.OffsetRange; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -54,7 +47,7 @@ public void setUp() { kafkaTestUtils = new KafkaTestUtils(); kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); } @@ -73,19 +66,19 @@ public void tearDown() { @Test public void testKafkaStream() throws InterruptedException { - final String topic1 = "new_topic1_testKafkaDirectStream"; - final String topic2 = "new_topic2_testKafkaDirectStream"; + final String topic1 = "topic1_testKafkaDirectStream"; + final String topic2 = "topic2_testKafkaDirectStream"; // hold a reference to the current offset ranges, so it can be used downstream final AtomicReference offsetRanges = new AtomicReference<>(); String[] topic1data = createTopicAndSendData(topic1); String[] topic2data = createTopicAndSendData(topic2); - HashSet sent = new HashSet(); + Set sent = new HashSet<>(); sent.addAll(Arrays.asList(topic1data)); sent.addAll(Arrays.asList(topic2data)); - HashMap kafkaParams = new HashMap(); + Map kafkaParams = new HashMap<>(); kafkaParams.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaTestUtils.brokerAddress()); kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); @@ -102,17 +95,17 @@ public void testKafkaStream() throws InterruptedException { // Make sure you can get offset ranges from the rdd new Function, JavaPairRDD>() { @Override - public JavaPairRDD call(JavaPairRDD rdd) throws Exception { + public JavaPairRDD call(JavaPairRDD rdd) { OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); offsetRanges.set(offsets); - Assert.assertEquals(offsets[0].topic(), topic1); + Assert.assertEquals(topic1, offsets[0].topic()); return rdd; } } ).map( new Function, String>() { @Override - public String call(Tuple2 kv) throws Exception { + public String call(Tuple2 kv) { return kv._2(); } } @@ -124,7 +117,7 @@ public String call(Tuple2 kv) throws Exception { String.class, String.class, kafkaParams, - topicOffsetToMap(topic2, (long) 0), + topicOffsetToMap(topic2, 0L), new Function, String>() { @Override public String call(ConsumerRecord consumerRecord) throws Exception { @@ -138,11 +131,11 @@ public String call(ConsumerRecord consumerRecord) throws Excepti unifiedStream.foreachRDD( new Function, Void>() { @Override - public Void call(JavaRDD rdd) throws Exception { + public Void call(JavaRDD rdd) { result.addAll(rdd.collect()); for (OffsetRange o : offsetRanges.get()) { System.out.println( - o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() + o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() ); } return null; @@ -160,14 +153,14 @@ public Void call(JavaRDD rdd) throws Exception { ssc.stop(); } - private HashSet topicToSet(String topic) { - HashSet topicSet = new HashSet(); + private static Set topicToSet(String topic) { + Set topicSet = new HashSet<>(); topicSet.add(topic); return topicSet; } - private HashMap topicOffsetToMap(String topic, Long offsetToStart) { - HashMap topicMap = new HashMap(); + private static Map topicOffsetToMap(String topic, Long offsetToStart) { + Map topicMap = new HashMap<>(); topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); return topicMap; } diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java index 1d79fcea3738..e020e436518f 100644 --- a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java +++ b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaRDDSuite.java @@ -19,18 +19,12 @@ import java.io.Serializable; import java.util.HashMap; +import java.util.Map; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.spark.streaming.kafka.v09.Broker; -import org.apache.spark.streaming.kafka.v09.KafkaTestUtils; -import org.apache.spark.streaming.kafka.v09.KafkaUtils; -import org.apache.spark.streaming.kafka.v09.OffsetRange; import scala.Tuple2; -import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -50,7 +44,7 @@ public void setUp() { kafkaTestUtils = new KafkaTestUtils(); kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); sc = new JavaSparkContext(sparkConf); } @@ -69,30 +63,23 @@ public void tearDown() { @Test public void testKafkaRDD() throws InterruptedException { - String topic1 = "new_topic1_testKafkaRDD"; - String topic2 = "new_topic2_testKafkaRDD"; + String topic1 = "topic1_testKafkaRDD"; + String topic2 = "topic2_testKafkaRDD"; - String[] topic1data = createTopicAndSendData(topic1); - String[] topic2data = createTopicAndSendData(topic2); + createTopicAndSendData(topic1); + createTopicAndSendData(topic2); - HashMap kafkaParams = new HashMap(); + Map kafkaParams = new HashMap<>(); kafkaParams.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaTestUtils.brokerAddress()); kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); kafkaParams.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); kafkaParams.put("spark.kafka.poll.time", "1000"); OffsetRange[] offsetRanges = { - OffsetRange.create(topic1, 0, 0, 1), - OffsetRange.create(topic2, 0, 0, 1) + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) }; - HashMap emptyLeaders = new HashMap(); - HashMap leaders = new HashMap(); - String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); - Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); - leaders.put(new TopicAndPartition(topic1, 0), broker); - leaders.put(new TopicAndPartition(topic2, 0), broker); - JavaRDD rdd1 = KafkaUtils.createRDD( sc, String.class, @@ -102,7 +89,7 @@ public void testKafkaRDD() throws InterruptedException { ).map( new Function, String>() { @Override - public String call(Tuple2 kv) throws Exception { + public String call(Tuple2 kv) { return kv._2(); } } diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java index ab98f2bd90dd..4a4dcf06e405 100644 --- a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java +++ b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java @@ -21,11 +21,8 @@ import java.util.*; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.spark.streaming.kafka.v09.KafkaTestUtils; -import org.apache.spark.streaming.kafka.v09.KafkaUtils; import scala.Tuple2; -import kafka.serializer.StringDecoder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -50,7 +47,7 @@ public void setUp() { kafkaTestUtils = new KafkaTestUtils(); kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); ssc = new JavaStreamingContext(sparkConf, new Duration(500)); } @@ -69,11 +66,11 @@ public void tearDown() { @Test public void testKafkaStream() throws InterruptedException { - String topic = "new_topic1_testKafkaStream"; - HashMap topics = new HashMap(); + String topic = "topic1_testKafkaStream"; + Map topics = new HashMap<>(); topics.put(topic, 1); - HashMap sent = new HashMap(); + Map sent = new HashMap<>(); sent.put("a", 5); sent.put("b", 3); sent.put("c", 10); @@ -81,7 +78,7 @@ public void testKafkaStream() throws InterruptedException { kafkaTestUtils.createTopic(topic); kafkaTestUtils.sendMessages(topic, sent); - HashMap kafkaParams = new HashMap(); + Map kafkaParams = new HashMap<>(); kafkaParams.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaTestUtils.brokerAddress()); kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, "test-consumer-" + random.nextInt(10000)); kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); @@ -90,39 +87,39 @@ public void testKafkaStream() throws InterruptedException { kafkaParams.put("spark.kafka.poll.time", "1000"); JavaPairDStream stream = KafkaUtils.createStream(ssc, - String.class, - String.class, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY_SER()); + String.class, + String.class, + kafkaParams, + topics, + StorageLevel.MEMORY_ONLY_SER()); final Map result = Collections.synchronizedMap(new HashMap()); JavaDStream words = stream.map( - new Function, String>() { - @Override - public String call(Tuple2 tuple2) throws Exception { - return tuple2._2(); - } + new Function, String>() { + @Override + public String call(Tuple2 tuple2) { + return tuple2._2(); } + } ); words.countByValue().foreachRDD( - new Function, Void>() { - @Override - public Void call(JavaPairRDD rdd) throws Exception { - List> ret = rdd.collect(); - for (Tuple2 r : ret) { - if (result.containsKey(r._1())) { - result.put(r._1(), result.get(r._1()) + r._2()); - } else { - result.put(r._1(), r._2()); - } + new Function, Void>() { + @Override + public Void call(JavaPairRDD rdd) { + List> ret = rdd.collect(); + for (Tuple2 r : ret) { + if (result.containsKey(r._1())) { + result.put(r._1(), result.get(r._1()) + r._2()); + } else { + result.put(r._1(), r._2()); } - - return null; } + + return null; } + } ); ssc.start(); @@ -134,8 +131,8 @@ public Void call(JavaPairRDD rdd) throws Exception { Thread.sleep(200); } Assert.assertEquals(sent.size(), result.size()); - for (String k : sent.keySet()) { - Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); + for (Map.Entry e : sent.entrySet()) { + Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); } } } From d7c631538538bf24f8a17b21db1bfd012214de41 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Mon, 4 Jan 2016 21:22:33 +0200 Subject: [PATCH 10/16] Make kafka consumer transient --- .../streaming/kafka/v09/KafkaCluster.scala | 16 +++--- .../spark/streaming/kafka/v09/KafkaRDD.scala | 3 +- .../kafka/v09/DirectKafkaStreamSuite.scala | 50 +++++++------------ .../kafka/v09/ReliableKafkaStreamSuite.scala | 4 ++ 4 files changed, 34 insertions(+), 39 deletions(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala index 5d1885f61a69..082955411753 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala @@ -41,6 +41,9 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String import KafkaCluster.{toTopicPart} + @transient + var consumer: KafkaConsumer[K, V] = null + def getLatestOffsets(topicPartitions: Set[TopicAndPartition]): Map[TopicAndPartition, Long] = { seek(topicPartitions, OffsetResetStrategy.LATEST) } @@ -134,18 +137,17 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String } private def withConsumer(fn: KafkaConsumer[K, V] => Any): Any = { - var consumer: KafkaConsumer[K, V] = null - try { + if (consumer == null) { consumer = new KafkaConsumer[K, V](kafkaParams.asInstanceOf[Map[String, Object]].asJava) - fn(consumer) - } finally { - if (consumer != null) { - consumer.close() - } } + fn(consumer) } def close(): Unit = { + if (consumer != null) { + consumer.close() + consumer = null + } } } diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala index 5e08c8d2c02f..d563bbc879d1 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala @@ -52,10 +52,11 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( private val KAFKA_DEFAULT_POLL_TIME: String = "0" private val pollTime = kafkaParams.get("spark.kafka.poll.time") .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt + private val cluster = new KafkaCluster[K, V](kafkaParams) override def getPartitions: Array[Partition] = { val topics = offsetRanges.map { _.topic }.toSet - val tpLeaders = new KafkaCluster[K, V](kafkaParams).getPartitionsLeader(topics) + val tpLeaders = cluster.getPartitionsLeader(topics) offsetRanges.zipWithIndex.map { case (o, i) => diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala index 7f6697a1382a..680962f7ee64 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala @@ -175,12 +175,11 @@ class DirectKafkaStreamSuite assert( stream.asInstanceOf[DirectKafkaInputDStream[_, _, _]] .fromOffsets(topicPartition) >= offsetBeforeStart, - "Start offset not from latest") + "Start offset not from latest" + ) val collectedData = new mutable.ArrayBuffer[String]() with mutable.SynchronizedBuffer[String] - stream.map { - _._2 - }.foreachRDD { rdd => collectedData ++= rdd.collect() } + stream.map { _._2 }.foreachRDD { rdd => collectedData ++= rdd.collect() } ssc.start() val newData = Map("b" -> 10) kafkaTestUtils.sendMessages(topic, newData) @@ -255,13 +254,9 @@ class DirectKafkaStreamSuite // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { - val strings = data.map { - _.toString - } - kafkaTestUtils.sendMessages(topic, strings.map { - _ -> 1 - }.toMap) - eventually(timeout(20 seconds), interval(50 milliseconds)) { + val strings = data.map { _.toString } + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1 }.toMap) + eventually(timeout(60 seconds), interval(200 milliseconds)) { assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) @@ -282,9 +277,7 @@ class DirectKafkaStreamSuite // This is to collect the raw data received from Kafka kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => - val data = rdd.map { - _._2 - }.collect() + val data = rdd.map { _._2 }.collect() DirectKafkaStreamSuite.collectedData.appendAll(data) } @@ -303,10 +296,9 @@ class DirectKafkaStreamSuite val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") assert( - offsetRangesBeforeStop.head._2.forall { - _.fromOffset === 0 - }, - "starting offset not zero") + offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, + "starting offset not zero" + ) ssc.stop() logInfo("====== RESTARTING ========") @@ -322,7 +314,8 @@ class DirectKafkaStreamSuite recoveredOffsetRanges.forall { or => earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) }, - "Recovered ranges are not the same as the ones generated") + "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() @@ -425,9 +418,7 @@ class DirectKafkaStreamSuite // This is to collect the raw data received from Kafka kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => - val data = rdd.map { - _._2 - }.collect() + val data = rdd.map { _._2 }.collect() collectedData += data } @@ -441,7 +432,7 @@ class DirectKafkaStreamSuite // Expect blocks of data equal to "rate", scaled by the interval length in secs. val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) kafkaTestUtils.sendMessages(topic, messages) - eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { + eventually(timeout(10.seconds), interval(batchIntervalMilliseconds.milliseconds)) { // Assert that rate estimator values are used to determine maxMessagesPerPartition. // Funky "-" in message makes the complete assertion message read better. assert(collectedData.exists(_.size == expectedSize), @@ -457,9 +448,7 @@ class DirectKafkaStreamSuite Seq[(Time, Array[OffsetRange])] = { kafkaStream.generatedRDDs.mapValues { rdd => rdd.asInstanceOf[KafkaRDD[K, V, (K, V)]].offsetRanges - }.toSeq.sortBy { - _._1 - } + }.toSeq.sortBy { _._1 } } } @@ -484,7 +473,6 @@ object DirectKafkaStreamSuite { numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) } } - } private[streaming] class ConstantEstimator(@volatile private var rate: Long) @@ -495,9 +483,9 @@ private[streaming] class ConstantEstimator(@volatile private var rate: Long) } def compute( - time: Long, - elements: Long, - processingDelay: Long, - schedulingDelay: Long): Option[Double] = Some(rate) + time: Long, + elements: Long, + processingDelay: Long, + schedulingDelay: Long): Option[Double] = Some(rate) } diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala index 235d10869c5d..540ab63ebc92 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala @@ -94,6 +94,10 @@ class ReliableKafkaStreamSuite extends SparkFunSuite ssc.stop() ssc = null } + if (kc != null) { + kc.close() + kc = null + } } test("Reliable Kafka input stream with single topic") { From 8510585ed253f71e364695c17437ca37d0ba1ff8 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Fri, 8 Jan 2016 16:37:38 +0200 Subject: [PATCH 11/16] Add LeaderOffset to KafkaCluster and change TopicAndPartition usage to TopicPartition --- .../kafka/v09/DirectKafkaInputDStream.scala | 32 +++--- .../streaming/kafka/v09/KafkaCluster.scala | 97 +++++++++++-------- .../spark/streaming/kafka/v09/KafkaRDD.scala | 29 +++--- .../streaming/kafka/v09/KafkaUtils.scala | 15 +-- .../streaming/kafka/v09/OffsetRange.scala | 28 +++++- .../kafka/v09/ReliableKafkaReceiver.scala | 41 ++++---- .../kafka/v09/JavaDirectKafkaStreamSuite.java | 8 +- .../kafka/v09/DirectKafkaStreamSuite.scala | 7 +- .../kafka/v09/KafkaClusterSuite.scala | 3 +- .../streaming/kafka/v09/KafkaRDDSuite.scala | 5 +- .../kafka/v09/ReliableKafkaStreamSuite.scala | 3 +- 11 files changed, 160 insertions(+), 108 deletions(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala index bb7e179f41d0..448baf867795 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaInputDStream.scala @@ -19,8 +19,10 @@ package org.apache.spark.streaming.kafka.v09 import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.ConsumerRecord +import org.apache.kafka.common.TopicPartition import org.apache.spark.Logging import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.kafka.v09.KafkaCluster.LeaderOffset import org.apache.spark.streaming.scheduler.rate.RateEstimator import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} import org.apache.spark.streaming.{StreamingContext, Time} @@ -52,7 +54,7 @@ class DirectKafkaInputDStream[ R: ClassTag]( @transient ssc_ : StreamingContext, val kafkaParams: Map[String, String], - val fromOffsets: Map[TopicAndPartition, Long], + @transient val fromOffsets: Map[TopicPartition, Long], messageHandler: ConsumerRecord[K, V] => R ) extends InputDStream[R](ssc_) with Logging { @@ -105,32 +107,39 @@ class DirectKafkaInputDStream[ } } - protected var currentOffsets = fromOffsets + // temp fix for serialization issue of TopicPartition + protected var serCurrentOffsets = fromOffsets.map { case(tp, l) => + (tp.topic, tp.partition, l); + } + + @transient + protected var currentOffsets: Map[TopicPartition, Long] = null - protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, Long] = { - val topicPartition = fromOffsets.map(fo => fo._1).toSet - kafkaCluster.getLatestOffsets(topicPartition) + protected final def latestLeaderOffsets(): Map[TopicPartition, LeaderOffset] = { + kafkaCluster.getLatestOffsetsWithLeaders(currentOffsets.keySet) } // limits the maximum number of messages per partition - protected def clamp(leaderOffsets: Map[TopicAndPartition, Long] - ): Map[TopicAndPartition, Long] = { + protected def clamp( + leaderOffsets: Map[TopicPartition, LeaderOffset] + ): Map[TopicPartition, LeaderOffset] = { maxMessagesPerPartition.map { mmp => leaderOffsets.map { case (tp, lo) => - tp -> (Math.min(currentOffsets(tp) + mmp, lo)) + tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset)) } }.getOrElse(leaderOffsets) } override def compute(validTime: Time): Option[KafkaRDD[K, V, R]] = { - val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) + currentOffsets = serCurrentOffsets.map { i => new TopicPartition(i._1, i._2) -> i._3 }.toMap + val untilOffsets = clamp(latestLeaderOffsets()) val rdd = KafkaRDD[K, V, R]( context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) // Report the record number and metadata of this batch interval to InputInfoTracker. val offsetRanges = currentOffsets.map { case (tp, fo) => val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) } val description = offsetRanges.filter { offsetRange => // Don't display empty ranges. @@ -146,7 +155,7 @@ class DirectKafkaInputDStream[ val inputInfo = StreamInputInfo(id, rdd.count, metadata) ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) - currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2) + serCurrentOffsets = untilOffsets.map { kv => (kv._1.topic, kv._1.partition, kv._2.offset) } Some(rdd) } @@ -178,7 +187,6 @@ class DirectKafkaInputDStream[ override def restore() { // this is assuming that the topics don't change during execution, which is true currently - val topics = fromOffsets.keySet batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala index 082955411753..c4ff717329d6 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaCluster.scala @@ -20,14 +20,12 @@ package org.apache.spark.streaming.kafka.v09 import java.util import java.util.{Collections} -import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.{OffsetResetStrategy, KafkaConsumer, OffsetAndMetadata} import org.apache.kafka.common.{PartitionInfo, TopicPartition} import org.apache.spark.SparkException import scala.collection.JavaConverters._ import scala.reflect._ -import scala.util.control.NonFatal /** * @param kafkaParams Kafka @@ -39,20 +37,20 @@ private[spark] class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String]) extends Serializable { - import KafkaCluster.{toTopicPart} + import KafkaCluster.LeaderOffset @transient - var consumer: KafkaConsumer[K, V] = null + protected var consumer: KafkaConsumer[K, V] = null - def getLatestOffsets(topicPartitions: Set[TopicAndPartition]): Map[TopicAndPartition, Long] = { - seek(topicPartitions, OffsetResetStrategy.LATEST) + def getLatestOffsets(topicPartitions: Set[TopicPartition]): Map[TopicPartition, Long] = { + getOffsetsWithoutLeaders(topicPartitions, OffsetResetStrategy.LATEST) } - def getEarliestOffsets(topicPartitions: Set[TopicAndPartition]): Map[TopicAndPartition, Long] = { - seek(topicPartitions, OffsetResetStrategy.EARLIEST) + def getEarliestOffsets(topicPartitions: Set[TopicPartition]): Map[TopicPartition, Long] = { + getOffsetsWithoutLeaders(topicPartitions, OffsetResetStrategy.EARLIEST) } - def getPartitions(topics: Set[String]): Set[TopicAndPartition] = { + def getPartitions(topics: Set[String]): Set[TopicPartition] = { withConsumer { consumer => { val partInfo = topics.flatMap { topic => Option(consumer.partitionsFor(topic)) match { @@ -60,12 +58,12 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String case Some(partInfoList) => partInfoList.asScala.toList } } - val topicPartitions: Set[TopicAndPartition] = partInfo.map { partition => - new TopicAndPartition(partition.topic(), partition.partition()) + val topicPartitions: Set[TopicPartition] = partInfo.map { partition => + new TopicPartition(partition.topic(), partition.partition()) } topicPartitions } - }.asInstanceOf[Set[TopicAndPartition]] + }.asInstanceOf[Set[TopicPartition]] } def getPartitionsLeader(topics: Set[String]): Map[TopicPartition, String] = { @@ -86,16 +84,16 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String }.asInstanceOf[Set[PartitionInfo]] } - def setConsumerOffsets(offsets: Map[TopicAndPartition, Long]): Unit = { + def setConsumerOffsets(offsets: Map[TopicPartition, Long]): Unit = { val topicPartOffsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() - val topicAndPartition = offsets.map(tpl => tpl._1).toSeq + val topicPartition = offsets.map(tpl => tpl._1).toSeq withConsumer(consumer => { consumer.assign(Collections.emptyList[TopicPartition]) - consumer.assign(topicAndPartition.map(tp => toTopicPart(tp)).asJava) + consumer.assign(topicPartition.asJava) for ((topicAndPart, offset) <- offsets) { - val topicPartition = toTopicPart(topicAndPart) + val topicPartition = topicAndPart val offsetAndMetadata = new OffsetAndMetadata(offset) topicPartOffsets.put(topicPartition, offsetAndMetadata) } @@ -104,36 +102,61 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String }) } - def getCommittedOffsets(topicAndPartitions: Set[TopicAndPartition]): - Map[TopicAndPartition, Long] = { + def getCommittedOffsets(topicPartitions: Set[TopicPartition]): + Map[TopicPartition, Long] = { withConsumer(consumer => { - val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) consumer.assign(topicPartitions.toList.asJava) - topicAndPartitions.map( tp => { - val offsetAndMetadata = consumer.committed(toTopicPart(tp)) + topicPartitions.map( tp => { + val offsetAndMetadata = consumer.committed(tp) Option(offsetAndMetadata) match { case None => throw new SparkException(s"Topic $tp hasn't committed offsets") case Some(om) => tp -> om.offset() } } ).toMap - }).asInstanceOf[Map[TopicAndPartition, Long]] + }).asInstanceOf[Map[TopicPartition, Long]] } - def seek(topicAndPartitions: Set[TopicAndPartition], resetStrategy: OffsetResetStrategy): - Map[TopicAndPartition, Long] = { - withConsumer(consumer => { - val topicPartitions = topicAndPartitions.map(tp => toTopicPart(tp)) + def getLatestOffsetsWithLeaders( + topicPartitions: Set[TopicPartition] + ): Map[TopicPartition, LeaderOffset] = { + getOffsets(topicPartitions, OffsetResetStrategy.LATEST) + } + + private def getOffsetsWithoutLeaders( + topicPartitions: Set[TopicPartition], + offsetResetType: OffsetResetStrategy + ): Map[TopicPartition, Long] = { + getOffsets(topicPartitions, offsetResetType) + .map { t => (t._1, t._2.offset) } + } + + def getOffsets(topicPartitions: Set[TopicPartition], resetStrategy: OffsetResetStrategy): + Map[TopicPartition, LeaderOffset] = { + val topics = topicPartitions.map { _.topic } + withConsumer{ consumer => + val tplMap = topics.flatMap { topic => + Option(consumer.partitionsFor(topic)) match { + case None => + throw new SparkException("Topic doesnt exist " + topic) + case Some(piList) => piList.asScala.toList + } + }.map { pi => + new TopicPartition(pi.topic, pi.partition) -> pi.leader.host + }.toMap + consumer.assign(topicPartitions.toList.asJava) resetStrategy match { - case OffsetResetStrategy.EARLIEST => consumer.seekToBeginning(topicPartitions.toArray: _*) - case OffsetResetStrategy.LATEST => consumer.seekToEnd(topicPartitions.toArray: _*) + case OffsetResetStrategy.EARLIEST => consumer.seekToBeginning(topicPartitions.toList: _*) + case OffsetResetStrategy.LATEST => consumer.seekToEnd(topicPartitions.toList: _*) case _ => throw new SparkException("Unknown OffsetResetStrategy " + resetStrategy) } - topicAndPartitions.map( - tp => tp -> (consumer.position(toTopicPart(tp))) - ).toMap - }).asInstanceOf[Map[TopicAndPartition, Long]] + topicPartitions.map { tp => + val pos = consumer.position(tp) + tp -> new LeaderOffset(tplMap(tp), pos) + }.toMap + + }.asInstanceOf[Map[TopicPartition, LeaderOffset]] } private def withConsumer(fn: KafkaConsumer[K, V] => Any): Any = { @@ -152,13 +175,9 @@ class KafkaCluster[K: ClassTag, V: ClassTag](val kafkaParams: Map[String, String } +private[spark] object KafkaCluster { - def toTopicPart(topicAndPartition: TopicAndPartition): TopicPartition = { - new TopicPartition(topicAndPartition.topic, topicAndPartition.partition) - } - - def toTopicAndPart(topicPartition: TopicPartition): TopicAndPartition = { - TopicAndPartition(topicPartition.topic, topicPartition.partition) - } + private[spark] + case class LeaderOffset(host: String, offset: Long) } diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala index d563bbc879d1..95c7b4922368 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala @@ -19,13 +19,11 @@ package org.apache.spark.streaming.kafka.v09 import java.util.{ Collections, Properties } -import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.{ ConsumerRecord, KafkaConsumer } import org.apache.kafka.common.TopicPartition import org.apache.spark.partial.{ BoundedDouble, PartialResult } import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.kafka.v09.KafkaCluster -import org.apache.spark.streaming.kafka.v09.KafkaCluster.toTopicPart +import org.apache.spark.streaming.kafka.v09.KafkaCluster.{LeaderOffset} import org.apache.spark.util.NextIterator import org.apache.spark.{ Logging, Partition, SparkContext, TaskContext } @@ -55,13 +53,9 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( private val cluster = new KafkaCluster[K, V](kafkaParams) override def getPartitions: Array[Partition] = { - val topics = offsetRanges.map { _.topic }.toSet - val tpLeaders = cluster.getPartitionsLeader(topics) - offsetRanges.zipWithIndex.map { case (o, i) => - new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, - tpLeaders(new TopicPartition(o.topic, o.partition))) + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, o.leaderHost) }.toArray } @@ -108,7 +102,12 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( override def getPreferredLocations(thePart: Partition): Seq[String] = { val part = thePart.asInstanceOf[KafkaRDDPartition] // TODO is additional hostname resolution necessary here - Seq(part.host) + if (part.host != null ) { + Seq(part.host) + } + else { + Seq() + } } private def errBeginAfterEnd(part: KafkaRDDPartition): String = @@ -151,12 +150,12 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( kafkaParams.foreach(param => props.put(param._1, param._2)) val consumer = new KafkaConsumer[K, V](props) - val tp = new TopicAndPartition(part.topic, part.partition) - consumer.assign(Collections.singletonList[TopicPartition](toTopicPart(tp))) + val tp = new TopicPartition(part.topic, part.partition) + consumer.assign(Collections.singletonList[TopicPartition](tp)) var requestOffset = part.fromOffset var iter: java.util.Iterator[ConsumerRecord[K, V]] = null - consumer.seek(toTopicPart(tp), requestOffset) + consumer.seek(tp, requestOffset) override def close(): Unit = { if (consumer != null) { @@ -205,13 +204,13 @@ object KafkaRDD { def apply[K: ClassTag, V: ClassTag, R: ClassTag]( sc: SparkContext, kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - untilOffsets: Map[TopicAndPartition, Long], + fromOffsets: Map[TopicPartition, Long], + untilOffsets: Map[TopicPartition, LeaderOffset], messageHandler: ConsumerRecord[K, V] => R): KafkaRDD[K, V, R] = { val offsetRanges = fromOffsets.map { case (tp, fo) => val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo) + OffsetRange(tp.topic, tp.partition, fo, uo.offset, uo.host) }.toArray new KafkaRDD[K, V, R](sc, kafkaParams, offsetRanges, messageHandler) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala index 9ae8afcc5eac..990a988d62a7 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala @@ -22,6 +22,7 @@ import java.lang.{Integer => JInt, Long => JLong} import java.util.{Map => JMap, Set => JSet} import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.config.SslConfigs import scala.reflect.ClassTag @@ -156,14 +157,14 @@ object KafkaUtils { /** Make sure offsets are available in kafka, or throw an exception */ private def checkOffsets( - kc: KafkaCluster[_, _], - offsetRanges: Array[OffsetRange]): Unit = { - val topics = offsetRanges.map(_.topicAndPartition).toSet + kc: KafkaCluster[_, _], + offsetRanges: Array[OffsetRange]): Unit = { + val topics = offsetRanges.map(_.topicPartition).toSet val low = kc.getEarliestOffsets(topics) val high = kc.getLatestOffsets(topics) val badRanges = offsetRanges.filterNot { o => - low(o.topicAndPartition()) <= o.fromOffset && - o.untilOffset <= high(o.topicAndPartition()) + low(o.topicPartition()) <= o.fromOffset && + o.untilOffset <= high(o.topicPartition()) } if (!badRanges.isEmpty) { throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) @@ -292,7 +293,7 @@ object KafkaUtils { def createDirectStream[K: ClassTag, V: ClassTag, R: ClassTag]( ssc: StreamingContext, kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], + fromOffsets: Map[TopicPartition, Long], messageHandler: ConsumerRecord[K, V] => R ): InputDStream[R] = { val cleanedHandler = ssc.sc.clean(messageHandler) @@ -388,7 +389,7 @@ object KafkaUtils { valueClass: Class[V], recordClass: Class[R], kafkaParams: JMap[String, String], - fromOffsets: JMap[TopicAndPartition, JLong], + fromOffsets: JMap[TopicPartition, JLong], messageHandler: JFunction[ConsumerRecord[K, V], R] ): JavaInputDStream[R] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala index ff245d3df4b6..20672ac4745c 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/OffsetRange.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.kafka.v09 import kafka.common.TopicAndPartition +import org.apache.kafka.common.TopicPartition /** * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the @@ -46,12 +47,24 @@ final class OffsetRange private( val topic: String, val partition: Int, val fromOffset: Long, - val untilOffset: Long) extends Serializable { + val untilOffset: Long, + val leaderHost: String) extends Serializable { import OffsetRange.OffsetRangeTuple + def this( + topic: String, + partition: Int, + fromOffset: Long, + untilOffset: Long + ) = { + this(topic, partition, fromOffset, untilOffset, null) + } + /** Kafka TopicAndPartition object, for convenience */ def topicAndPartition(): TopicAndPartition = TopicAndPartition(topic, partition) + def topicPartition(): TopicPartition = new TopicPartition(topic, partition) + /** Number of messages this OffsetRange refers to */ def count(): Long = untilOffset - fromOffset @@ -93,11 +106,20 @@ object OffsetRange { def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = new OffsetRange(topic, partition, fromOffset, untilOffset) + def apply( - topicAndPartition: TopicAndPartition, + topic: String, + partition: Int, + fromOffset: Long, + untilOffset: Long, + leaderHost: String): OffsetRange = + new OffsetRange(topic, partition, fromOffset, untilOffset, leaderHost) + + def apply( + topicPartition: TopicPartition, fromOffset: Long, untilOffset: Long): OffsetRange = - new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + new OffsetRange(topicPartition.topic, topicPartition.partition, fromOffset, untilOffset) /** this is to avoid ClassNotFoundException during checkpoint restore */ private[kafka] diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala index fcec93244f8b..7e2e0b679e82 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala @@ -71,10 +71,10 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( * A HashMap to manage the offset for each topic/partition, this HashMap is called in * synchronized block, so mutable HashMap will not meet concurrency issue. */ - private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null + private var topicPartitionOffsetMap: mutable.HashMap[TopicPartition, Long] = null /** A concurrent HashMap to store the stream block id and related offset snapshot. */ - private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null + private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicPartition, Long]] = null /** * Manage the BlockGenerator in receiver itself for better managing block store and offset @@ -86,7 +86,7 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( private var messageHandlerThreadPool: ThreadPoolExecutor = null private var topicAndPartitionConsumerMap: - mutable.HashMap[TopicAndPartition, KafkaConsumer[K, V]] = null + mutable.HashMap[TopicPartition, KafkaConsumer[K, V]] = null private var consumerAndLockMap: mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] = null @@ -94,13 +94,13 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( override def onStart(): Unit = { logInfo(s"Starting Kafka Consumer Stream with group: $groupId") // Initialize the topic-partition / offset hash map. - topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] + topicPartitionOffsetMap = new mutable.HashMap[TopicPartition, Long] - topicAndPartitionConsumerMap = new mutable.HashMap[TopicAndPartition, KafkaConsumer[K, V]] + topicAndPartitionConsumerMap = new mutable.HashMap[TopicPartition, KafkaConsumer[K, V]] consumerAndLockMap = new mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] // Initialize the stream block id / offset snapshot hash map. - blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]]() + blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicPartition, Long]]() // Initialize the block generator for storing Kafka message. blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) @@ -126,17 +126,17 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( try { // Start the messages handler for each partition - val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet) - val iter = topicAndPartitions.iterator + val topicPartitions = kafkaCluster.getPartitions(topics.keys.toSet) + val iter = topicPartitions.iterator while (iter.hasNext) { - val topicAndPartition = iter.next() + val topicPartition = iter.next() val newConsumer = new KafkaConsumer[K, V](props) topicAndPartitionConsumerMap.put( - topicAndPartition, + topicPartition, newConsumer ) consumerAndLockMap.put(newConsumer, new ReentrantLock(true)) - newConsumer.subscribe(Collections.singletonList[String](topicAndPartition.topic)) + newConsumer.subscribe(Collections.singletonList[String](topicPartition.topic)) messageHandlerThreadPool.submit(new MessageHandler(newConsumer)) } } finally { @@ -182,8 +182,8 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( } /** Update stored offset */ - private def updateOffset(topicAndPartition: TopicAndPartition, offset: Long): Unit = { - topicPartitionOffsetMap.put(topicAndPartition, offset) + private def updateOffset(topicPartition: TopicPartition, offset: Long): Unit = { + topicPartitionOffsetMap.put(topicPartition, offset) } /** @@ -229,15 +229,14 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( /** * Commit the offset of Kafka's topic/partition */ - private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = { + private def commitOffset(offsetMap: Map[TopicPartition, Long]): Unit = { val offsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() - for ((topicAndPart, offset) <- offsetMap) { - val kafkaConsumer = topicAndPartitionConsumerMap.getOrElse(topicAndPart, - throw new RuntimeException(s"Failed to get consumer for $topicAndPart")) + for ((topicPart, offset) <- offsetMap) { + val kafkaConsumer = topicAndPartitionConsumerMap.getOrElse(topicPart, + throw new RuntimeException(s"Failed to get consumer for $topicPart")) - val topicPartition = new TopicPartition(topicAndPart.topic, topicAndPart.partition) val offsetAndMetadata = new OffsetAndMetadata(offset) - offsets.put(topicPartition, offsetAndMetadata) + offsets.put(topicPart, offsetAndMetadata) val lock = consumerAndLockMap(kafkaConsumer) lock.lock() try { @@ -285,8 +284,8 @@ class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( def onAddData(data: Any, metadata: Any): Unit = { // Update the offset of the data that was added to the generator if (metadata != null) { - val (topicAndPartition, offset) = metadata.asInstanceOf[(TopicAndPartition, Long)] - updateOffset(topicAndPartition, offset) + val (topicPartition, offset) = metadata.asInstanceOf[(TopicPartition, Long)] + updateOffset(topicPartition, offset) } } diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java index e9ecc279230a..71e0e9203d64 100644 --- a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaDirectKafkaStreamSuite.java @@ -18,9 +18,9 @@ package org.apache.spark.streaming.kafka.v09; -import kafka.common.TopicAndPartition; import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.common.TopicPartition; import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -159,9 +159,9 @@ private static Set topicToSet(String topic) { return topicSet; } - private static Map topicOffsetToMap(String topic, Long offsetToStart) { - Map topicMap = new HashMap<>(); - topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); + private static Map topicOffsetToMap(String topic, Long offsetToStart) { + Map topicMap = new HashMap<>(); + topicMap.put(new TopicPartition(topic, 0), offsetToStart); return topicMap; } diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala index 680962f7ee64..0b181685dfc9 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicLong import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.{ ConsumerRecord, ConsumerConfig } +import org.apache.kafka.common.TopicPartition import org.apache.spark.rdd.RDD import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.scheduler.rate.RateEstimator @@ -143,7 +144,7 @@ class DirectKafkaStreamSuite test("receiving from latest starting offset") { val topic = "new_latest" - val topicPartition = TopicAndPartition(topic, 0) + val topicPartition = new TopicPartition(topic, 0) val data = Map("a" -> 10) kafkaTestUtils.createTopic(topic) val kafkaParams = Map( @@ -191,7 +192,7 @@ class DirectKafkaStreamSuite test("creating stream by offset") { val topic = "new_offset" - val topicPartition = TopicAndPartition(topic, 0) + val topicPartition = new TopicPartition(topic, 0) val data = Map("a" -> 10) kafkaTestUtils.createTopic(topic) val kafkaParams = Map( @@ -371,7 +372,7 @@ class DirectKafkaStreamSuite test("using rate controller") { val topic = "new_backpressure" - val topicPartition = TopicAndPartition(topic, 0) + val topicPartition = new TopicPartition(topic, 0) kafkaTestUtils.createTopic(topic) val kafkaParams = Map( ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala index 93293acd2b49..7b7c16456468 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaClusterSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.kafka.v09 import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkFunSuite import org.scalatest.BeforeAndAfterAll @@ -26,7 +27,7 @@ import scala.util.Random class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { private val topic = "new_kcsuitetopic" + Random.nextInt(10000) - private val topicPartition = TopicAndPartition(topic, 0) + private val topicPartition = new TopicPartition(topic, 0) private var newKc: KafkaCluster[_, _] = null private var kafkaTestUtils: KafkaTestUtils = _ diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala index 8b00e6a759f2..9a296f0f33ac 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaRDDSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.kafka.v09 import kafka.common.TopicAndPartition import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } +import org.apache.kafka.common.TopicPartition import org.apache.spark._ import org.scalatest.BeforeAndAfterAll @@ -128,7 +129,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { 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 + val rangesMap = ranges.map(o => new TopicPartition(o.topic, o.partition) -> o.untilOffset).toMap // make sure consumer offsets are committed before the next getRdd call kc.setConsumerOffsets(rangesMap) @@ -164,7 +165,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { } val latestOffsets = kc.getLatestOffsets(topicPartitions) - val offsetRanges = consumerOffsets.map { case (tp: TopicAndPartition, fromOffset: Long) => + val offsetRanges = consumerOffsets.map { case (tp: TopicPartition, fromOffset: Long) => OffsetRange(tp.topic, tp.partition, fromOffset, latestOffsets(tp)) }.toArray diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala index 540ab63ebc92..d5ee2ddb7d6a 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala @@ -22,6 +22,7 @@ import java.io.File import kafka.common.TopicAndPartition import kafka.utils.ZkUtils import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.TopicPartition import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{ Milliseconds, StreamingContext } import org.apache.spark.util.Utils @@ -157,7 +158,7 @@ class ReliableKafkaStreamSuite extends SparkFunSuite /** Getting partition offset from Zookeeper. */ private def getCommitOffset(topic: String, partition: Int): Option[Long] = { try { - val tp = TopicAndPartition(topic, partition) + val tp = new TopicPartition(topic, partition) Option(kc.getCommittedOffsets(Set(tp))(tp)) } catch { case e: SparkException => None From 2109b9445d8d31df0f84cadad8ed1f9ba201137c Mon Sep 17 00:00:00 2001 From: nikit-os Date: Fri, 8 Jan 2016 16:47:49 +0200 Subject: [PATCH 12/16] Remove receiver based InputStream --- .../kafka/v09/KafkaInputDStream.scala | 130 -------- .../streaming/kafka/v09/KafkaUtils.scala | 83 ----- .../kafka/v09/ReliableKafkaReceiver.scala | 307 ------------------ .../kafka/v09/JavaKafkaStreamSuite.java | 138 -------- .../kafka/v09/KafkaStreamSuite.scala | 88 ----- .../kafka/v09/ReliableKafkaStreamSuite.scala | 168 ---------- 6 files changed, 914 deletions(-) delete mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala delete mode 100644 external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala delete mode 100644 external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java delete mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala delete mode 100644 external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala deleted file mode 100644 index 0274e86db683..000000000000 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaInputDStream.scala +++ /dev/null @@ -1,130 +0,0 @@ -/* - * 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.v09 - -import java.util.{Collections, Properties} - -import org.apache.kafka.clients.consumer.{ConsumerRecord, ConsumerRecords, KafkaConsumer} -import org.apache.spark.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.ThreadUtils - -import scala.collection.Map -import scala.reflect.ClassTag - -/** - * Input stream that pulls messages from a Kafka Broker. - * - * @param kafkaParams Map of kafka configuration parameters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - */ -private[streaming] -class KafkaInputDStream[ -K: ClassTag, -V: ClassTag]( - @transient ssc_ : StreamingContext, - kafkaParams: Map[String, String], - topics: Map[String, Int], - useReliableReceiver: Boolean, - storageLevel: StorageLevel - ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { - - def getReceiver(): Receiver[(K, V)] = { - if (!useReliableReceiver) { - logInfo("[!] Using 0.9 KafkaReceiver") - new KafkaReceiver[K, V](kafkaParams, topics, storageLevel) - } else { - logInfo("[!] Using 0.9 ReliableKafkaReceiver") - new ReliableKafkaReceiver[K, V](kafkaParams, topics, storageLevel) - } - } -} - -private[streaming] -class KafkaReceiver[ -K: ClassTag, -V: ClassTag]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ) extends Receiver[(K, V)](storageLevel) with Logging { - - private var kafkaCluster: KafkaCluster[_, _] = null - - private val KAFKA_DEFAULT_POLL_TIME: String = "0" - private val pollTime = kafkaParams.get("spark.kafka.poll.time") - .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt - - def onStop() { - } - - def onStart() { - - logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) - - // Kafka connection properties - val props = new Properties() - kafkaParams.foreach(param => props.put(param._1, param._2)) - kafkaCluster = new KafkaCluster[K, V](kafkaParams.toMap) - - val executorPool = ThreadUtils.newDaemonFixedThreadPool(topics.values.sum, - "KafkaMessageHandler") - try { - // Start the messages handler for each partition - val topicAndPartitions = kafkaCluster.getPartitions(topics.keys.toSet) - val iter = topicAndPartitions.iterator - - while (iter.hasNext) { - val topicAndPartition = iter.next() - val newConsumer = new KafkaConsumer[K, V](props) - newConsumer.subscribe(Collections.singletonList[String](topicAndPartition.topic)) - executorPool.submit(new MessageHandler(newConsumer)) - } - } finally { - executorPool.shutdown() // Just causes threads to terminate after work is done - } - } - - // Handles Kafka messages - private class MessageHandler(consumer: KafkaConsumer[K, V]) - extends Runnable { - def run() { - logInfo("Starting MessageHandler.") - try { - while (true) { - val records: ConsumerRecords[K, V] = this.consumer.poll(pollTime) - val iterator = records.iterator() - while (iterator.hasNext) { - val record: ConsumerRecord[K, V] = iterator.next() - store((record.key, record.value())) - } - } - } catch { - case e: Throwable => reportError("Error handling message; exiting", e) - } - } - } - -} - diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala index 990a988d62a7..0dd856b01d28 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala @@ -33,7 +33,6 @@ import kafka.serializer.Decoder import net.razorvine.pickle.{Opcodes, Pickler, IObjectPickler} import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord} import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.streaming.util.WriteAheadLogUtils import org.apache.spark.{SSLOptions, SparkContext, SparkException} import scala.collection.JavaConverters._ @@ -73,88 +72,6 @@ object KafkaUtils { } - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param ssc StreamingContext object - * @param servers Broker servers (for Kafka 0.9) (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - */ - def createStream( - ssc: StreamingContext, - servers: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[(String, String)] = { - val kafkaParams = Map[String, String]( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> servers, - ConsumerConfig.GROUP_ID_CONFIG -> groupId, - ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG -> "5000", - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> - "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> - "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "1000" - ) - createStream[String, String]( - ssc, kafkaParams, topics, storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param ssc StreamingContext object - * @param kafkaParams Map of kafka configuration parameters - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - */ - def createStream[K: ClassTag, V: ClassTag]( - ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ): ReceiverInputDStream[(K, V)] = { - val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) - new KafkaInputDStream[K, V]( - ssc, - addSSLOptions(kafkaParams, ssc.sparkContext), - topics, - walEnabled, - storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param jssc JavaStreamingContext object - * @param keyTypeClass Key type of DStream - * @param valueTypeClass value type of Dstream - * @param kafkaParams Map of kafka configuration parameters - * @param topics Map of (topic_name -> numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel RDD storage level. - */ - def createStream[K, V]( - jssc: JavaStreamingContext, - keyTypeClass: Class[K], - valueTypeClass: Class[V], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) - - createStream[K, V]( - jssc.ssc, - Map(kafkaParams.asScala.toSeq: _*), - Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - /** Make sure offsets are available in kafka, or throw an exception */ private def checkOffsets( kc: KafkaCluster[_, _], diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala deleted file mode 100644 index 7e2e0b679e82..000000000000 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaReceiver.scala +++ /dev/null @@ -1,307 +0,0 @@ -/* - * 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.v09 - -import java.util -import java.util.concurrent.locks.ReentrantLock -import java.util.concurrent.{ConcurrentHashMap, ThreadPoolExecutor} -import java.util.{Collections, Properties} - -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import org.apache.kafka.clients.consumer._ -import org.apache.kafka.common.TopicPartition -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} -import org.apache.spark.util.ThreadUtils -import org.apache.spark.{Logging, SparkEnv} - -import scala.collection.{Map, mutable} -import scala.reflect.ClassTag - -/** - * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager - * without loss. - * It is turned off by default and will be enabled when - * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver - * is that this receiver manages topic-partition/offset itself and updates the offset information - * after data is reliably stored as write-ahead log. Offsets will only be updated when data is - * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated. - * - * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset - * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams - * will not take effect. - */ -private[streaming] -class ReliableKafkaReceiver[K: ClassTag, V: ClassTag]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel) extends Receiver[(K, V)](storageLevel) with Logging { - - private val groupId = kafkaParams("group.id") - private val AUTO_OFFSET_COMMIT = ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG - - private def conf = SparkEnv.get.conf - - private var kafkaCluster: KafkaCluster[_, _] = null - private val lock: ReentrantLock = new ReentrantLock() - - private var props: Properties = null - - private val KAFKA_DEFAULT_POLL_TIME: String = "0" - private val pollTime = kafkaParams.get("spark.kafka.poll.time") - .getOrElse(KAFKA_DEFAULT_POLL_TIME).toInt - - /** - * A HashMap to manage the offset for each topic/partition, this HashMap is called in - * synchronized block, so mutable HashMap will not meet concurrency issue. - */ - private var topicPartitionOffsetMap: mutable.HashMap[TopicPartition, Long] = null - - /** A concurrent HashMap to store the stream block id and related offset snapshot. */ - private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicPartition, Long]] = null - - /** - * Manage the BlockGenerator in receiver itself for better managing block store and offset - * commit. - */ - private var blockGenerator: BlockGenerator = null - - /** Thread pool running the handlers for receiving message from multiple topics and partitions. */ - private var messageHandlerThreadPool: ThreadPoolExecutor = null - - private var topicAndPartitionConsumerMap: - mutable.HashMap[TopicPartition, KafkaConsumer[K, V]] = null - - private var consumerAndLockMap: - mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] = null - - override def onStart(): Unit = { - logInfo(s"Starting Kafka Consumer Stream with group: $groupId") - // Initialize the topic-partition / offset hash map. - topicPartitionOffsetMap = new mutable.HashMap[TopicPartition, Long] - - topicAndPartitionConsumerMap = new mutable.HashMap[TopicPartition, KafkaConsumer[K, V]] - consumerAndLockMap = new mutable.HashMap[KafkaConsumer[K, V], ReentrantLock] - - // Initialize the stream block id / offset snapshot hash map. - blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicPartition, Long]]() - - // Initialize the block generator for storing Kafka message. - blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) - - if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { - logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + - "otherwise we will manually set it to false to turn off auto offset commit in Kafka") - } - - props = new Properties() - kafkaParams.foreach(param => props.put(param._1, param._2)) - // Manually set "auto.commit.enable" to "false" no matter user explicitly set it to true, - // we have to make sure this property is set to false to turn off auto commit mechanism in - // Kafka. - props.setProperty(AUTO_OFFSET_COMMIT, "false") - - messageHandlerThreadPool = ThreadUtils.newDaemonFixedThreadPool( - topics.values.sum, "KafkaMessageHandler") - - blockGenerator.start() - - kafkaCluster = new KafkaCluster[K, V](kafkaParams.toMap) - - try { - // Start the messages handler for each partition - val topicPartitions = kafkaCluster.getPartitions(topics.keys.toSet) - val iter = topicPartitions.iterator - while (iter.hasNext) { - val topicPartition = iter.next() - val newConsumer = new KafkaConsumer[K, V](props) - topicAndPartitionConsumerMap.put( - topicPartition, - newConsumer - ) - consumerAndLockMap.put(newConsumer, new ReentrantLock(true)) - newConsumer.subscribe(Collections.singletonList[String](topicPartition.topic)) - messageHandlerThreadPool.submit(new MessageHandler(newConsumer)) - } - } finally { - messageHandlerThreadPool.shutdown() // Just causes threads to terminate after work is done - } - } - - override def onStop(): Unit = { - if (messageHandlerThreadPool != null) { - messageHandlerThreadPool.shutdown() - messageHandlerThreadPool = null - } - - if (blockGenerator != null) { - blockGenerator.stop() - blockGenerator = null - } - - if (topicPartitionOffsetMap != null) { - topicPartitionOffsetMap.clear() - topicPartitionOffsetMap = null - } - - if (blockOffsetMap != null) { - blockOffsetMap.clear() - blockOffsetMap = null - } - } - - /** Store a Kafka message and the associated metadata as a tuple. */ - private def storeMessageAndMetadata(msgAndMetadata: MessageAndMetadata[K, V]): Unit = { - val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) - val data = (msgAndMetadata.key, msgAndMetadata.message) - val metadata = (topicAndPartition, msgAndMetadata.offset) - blockGenerator.addDataWithCallback(data, metadata) - } - - private def storeConsumerRecord(consumerRecord: ConsumerRecord[K, V]): Unit = { - val topicAndPartition = TopicAndPartition(consumerRecord.topic, consumerRecord.partition) - val data = (consumerRecord.key, consumerRecord.value()) - val metadata = (topicAndPartition, consumerRecord.offset) - blockGenerator.addDataWithCallback(data, metadata) - } - - /** Update stored offset */ - private def updateOffset(topicPartition: TopicPartition, offset: Long): Unit = { - topicPartitionOffsetMap.put(topicPartition, offset) - } - - /** - * Remember the current offsets for each topic and partition. This is called when a block is - * generated. - */ - private def rememberBlockOffsets(blockId: StreamBlockId): Unit = { - // Get a snapshot of current offset map and store with related block id. - val offsetSnapshot = topicPartitionOffsetMap.toMap - blockOffsetMap.put(blockId, offsetSnapshot) - topicPartitionOffsetMap.clear() - } - - /** - * Store the ready-to-be-stored block and commit the related offsets to Kafka. This method - * will try a fixed number of times to push the block. If the push fails, - * the receiver is stopped. - */ - private def storeBlockAndCommitOffset( - blockId: StreamBlockId, - arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - var count = 0 - var pushed = false - var exception: Exception = null - while (!pushed && count <= 3) { - try { - store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) - pushed = true - } catch { - case ex: Exception => - count += 1 - exception = ex - } - } - if (pushed) { - Option(blockOffsetMap.get(blockId)).foreach(commitOffset) - blockOffsetMap.remove(blockId) - } else { - stop("Error while storing block into Spark", exception) - } - } - - /** - * Commit the offset of Kafka's topic/partition - */ - private def commitOffset(offsetMap: Map[TopicPartition, Long]): Unit = { - val offsets = new util.HashMap[TopicPartition, OffsetAndMetadata]() - for ((topicPart, offset) <- offsetMap) { - val kafkaConsumer = topicAndPartitionConsumerMap.getOrElse(topicPart, - throw new RuntimeException(s"Failed to get consumer for $topicPart")) - - val offsetAndMetadata = new OffsetAndMetadata(offset) - offsets.put(topicPart, offsetAndMetadata) - val lock = consumerAndLockMap(kafkaConsumer) - lock.lock() - try { - kafkaConsumer.commitSync(offsets) - } finally { - lock.unlock() - } - } - - } - - /** Class to handle received Kafka message. */ - private final class MessageHandler(consumer: KafkaConsumer[K, V]) extends Runnable { - override def run(): Unit = { - var records: ConsumerRecords[K, V] = null - val lock = consumerAndLockMap(consumer) - while (!isStopped) { - try { - while (true) { - lock.lock() - try { - records = consumer.poll(pollTime) - } finally { - lock.unlock() - } - val iterator = records.iterator() - while (iterator.hasNext) { - val record: ConsumerRecord[K, V] = iterator.next() - storeConsumerRecord(record) - } - } - } catch { - case e: Exception => { - reportError("Error handling message", e) - } - - } - } - } - } - - /** Class to handle blocks generated by the block generator. */ - private final class GeneratedBlockHandler extends BlockGeneratorListener { - - def onAddData(data: Any, metadata: Any): Unit = { - // Update the offset of the data that was added to the generator - if (metadata != null) { - val (topicPartition, offset) = metadata.asInstanceOf[(TopicPartition, Long)] - updateOffset(topicPartition, offset) - } - } - - def onGenerateBlock(blockId: StreamBlockId): Unit = { - // Remember the offsets of topics/partitions when a block has been generated - rememberBlockOffsets(blockId) - } - - def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - // Store block and commit the blocks offset - storeBlockAndCommitOffset(blockId, arrayBuffer) - } - - def onError(message: String, throwable: Throwable): Unit = { - reportError(message, throwable) - } - } - -} diff --git a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java b/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java deleted file mode 100644 index 4a4dcf06e405..000000000000 --- a/external/kafka-v09/src/test/java/org/apache/spark/streaming/kafka/v09/JavaKafkaStreamSuite.java +++ /dev/null @@ -1,138 +0,0 @@ -/* - * 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.v09; - -import java.io.Serializable; -import java.util.*; - -import org.apache.kafka.clients.consumer.ConsumerConfig; -import scala.Tuple2; - -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.Duration; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaPairDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; - -public class JavaKafkaStreamSuite implements Serializable { - private transient JavaStreamingContext ssc = null; - private transient Random random = new Random(); - private transient KafkaTestUtils kafkaTestUtils = null; - - @Before - public void setUp() { - kafkaTestUtils = new KafkaTestUtils(); - kafkaTestUtils.setup(); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - ssc = new JavaStreamingContext(sparkConf, new Duration(500)); - } - - @After - public void tearDown() { - if (ssc != null) { - ssc.stop(); - ssc = null; - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown(); - kafkaTestUtils = null; - } - } - - @Test - public void testKafkaStream() throws InterruptedException { - String topic = "topic1_testKafkaStream"; - Map topics = new HashMap<>(); - topics.put(topic, 1); - - Map sent = new HashMap<>(); - sent.put("a", 5); - sent.put("b", 3); - sent.put("c", 10); - - kafkaTestUtils.createTopic(topic); - kafkaTestUtils.sendMessages(topic, sent); - - Map kafkaParams = new HashMap<>(); - kafkaParams.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafkaTestUtils.brokerAddress()); - kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, "test-consumer-" + random.nextInt(10000)); - kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); - kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); - kafkaParams.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringDeserializer"); - kafkaParams.put("spark.kafka.poll.time", "1000"); - - JavaPairDStream stream = KafkaUtils.createStream(ssc, - String.class, - String.class, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY_SER()); - - final Map result = Collections.synchronizedMap(new HashMap()); - - JavaDStream words = stream.map( - new Function, String>() { - @Override - public String call(Tuple2 tuple2) { - return tuple2._2(); - } - } - ); - - words.countByValue().foreachRDD( - new Function, Void>() { - @Override - public Void call(JavaPairRDD rdd) { - List> ret = rdd.collect(); - for (Tuple2 r : ret) { - if (result.containsKey(r._1())) { - result.put(r._1(), result.get(r._1()) + r._2()); - } else { - result.put(r._1(), r._2()); - } - } - - return null; - } - } - ); - - ssc.start(); - - long startTime = System.currentTimeMillis(); - boolean sizeMatches = false; - while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { - sizeMatches = sent.size() == result.size(); - Thread.sleep(200); - } - Assert.assertEquals(sent.size(), result.size()); - for (Map.Entry e : sent.entrySet()) { - Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); - } - } -} diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala deleted file mode 100644 index 17842f5ff77b..000000000000 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/KafkaStreamSuite.scala +++ /dev/null @@ -1,88 +0,0 @@ -/* - * 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.v09 - -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{ Milliseconds, StreamingContext } -import org.apache.spark.{ SparkConf, SparkFunSuite } -import org.scalatest.BeforeAndAfterAll -import org.scalatest.concurrent.Eventually - -import scala.collection.mutable -import scala.concurrent.duration._ -import scala.language.postfixOps -import scala.util.Random - -class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfterAll { - private var ssc: StreamingContext = _ - private var kafkaTestUtils: KafkaTestUtils = _ - - override def beforeAll(): Unit = { - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - } - - override def afterAll(): Unit = { - if (ssc != null) { - ssc.stop() - ssc = null - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } - - test("Kafka input stream") { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - ssc = new StreamingContext(sparkConf, Milliseconds(500)) - val topic = "new_topic_NewKafkaStreamSuite" - val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, sent) - - val kafkaParams = Map( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", - ConsumerConfig.GROUP_ID_CONFIG -> s"test-consumer-${Random.nextInt(10000)}", - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> - "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> - "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "100") - - val stream = KafkaUtils.createStream[String, String]( - ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) - val result = new mutable.HashMap[String, Long]() with mutable.SynchronizedMap[String, Long] - stream.map(_._2).countByValue().foreachRDD { r => - val ret = r.collect() - ret.toMap.foreach { kv => - val count = result.getOrElseUpdate(kv._1, 0) + kv._2 - result.put(kv._1, count) - } - } - - ssc.start() - - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - assert(sent === result) - } - } -} diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala deleted file mode 100644 index d5ee2ddb7d6a..000000000000 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/ReliableKafkaStreamSuite.scala +++ /dev/null @@ -1,168 +0,0 @@ -/* - * 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.v09 - -import java.io.File - -import kafka.common.TopicAndPartition -import kafka.utils.ZkUtils -import org.apache.kafka.clients.consumer.ConsumerConfig -import org.apache.kafka.common.TopicPartition -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{ Milliseconds, StreamingContext } -import org.apache.spark.util.Utils -import org.apache.spark.{SparkException, SparkConf, SparkFunSuite} -import org.scalatest.concurrent.Eventually -import org.scalatest.{ BeforeAndAfter, BeforeAndAfterAll } - -import scala.collection.mutable -import scala.concurrent.duration._ -import scala.language.postfixOps -import scala.util.Random - -class ReliableKafkaStreamSuite extends SparkFunSuite - with BeforeAndAfterAll with BeforeAndAfter with Eventually { - - private val sparkConf = new SparkConf() - .setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.receiver.writeAheadLog.enable", "true") - private val data = Map("a" -> 10, "b" -> 10, "c" -> 10) - - private var kafkaTestUtils: KafkaTestUtils = _ - - private var groupId: String = _ - private var kafkaParams: Map[String, String] = _ - private var ssc: StreamingContext = _ - private var tempDirectory: File = null - private var zkUtils: ZkUtils = null - - private var kc: KafkaCluster[_, _] = null - - override def beforeAll(): Unit = { - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - - groupId = s"new_test-consumer-${Random.nextInt(10000)}" - - kafkaParams = Map( - ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG -> kafkaTestUtils.brokerAddress, - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG -> "earliest", - ConsumerConfig.GROUP_ID_CONFIG -> groupId, - ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> - "org.apache.kafka.common.serialization.StringDeserializer", - ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG -> - "org.apache.kafka.common.serialization.StringDeserializer", - "spark.kafka.poll.time" -> "100") - - kc = new KafkaCluster(kafkaParams) - - tempDirectory = Utils.createTempDir() - zkUtils = ZkUtils(kafkaTestUtils.zookeeperClient, false) - } - - override def afterAll(): Unit = { - Utils.deleteRecursively(tempDirectory) - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } - - before { - ssc = new StreamingContext(sparkConf, Milliseconds(500)) - ssc.checkpoint(tempDirectory.getAbsolutePath) - } - - after { - if (ssc != null) { - ssc.stop() - ssc = null - } - if (kc != null) { - kc.close() - kc = null - } - } - - test("Reliable Kafka input stream with single topic") { - val topic = "test-topic" - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, data) - - // Verify whether the offset of this group/topic/partition is 0 before starting. - assert(getCommitOffset(topic, 0) === None) - - val stream = KafkaUtils.createStream[String, String]( - ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) - val result = new mutable.HashMap[String, Long]() - stream.map { case (k, v) => v }.foreachRDD { r => - val ret = r.collect() - ret.foreach { v => - val count = result.getOrElseUpdate(v, 0) + 1 - result.put(v, count) - } - } - ssc.start() - - eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { - // A basic process verification for ReliableKafkaReceiver. - // Verify whether received message number is equal to the sent message number. - assert(data.size === result.size) - // Verify whether each message is the same as the data to be verified. - data.keys.foreach { k => assert(data(k) === result(k).toInt) } - // Verify the offset number whether it is equal to the total message number. - assert(getCommitOffset(topic, 0) === Some(29L)) - } - } - - test("Reliable Kafka input stream with multiple topics") { - val topics = Map("new_topic1" -> 1, "new_topic2" -> 1, "new_topic3" -> 1) - topics.foreach { - case (t, _) => - kafkaTestUtils.createTopic(t) - kafkaTestUtils.sendMessages(t, data) - } - - // Before started, verify all the group/topic/partition offsets are 0. - // topics.foreach { case (t, _) => assert(getCommitOffset(t, 0) === None) } - - // Consuming all the data sent to the broker which will potential commit the offsets internally. - val stream = KafkaUtils.createStream[String, String]( - ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) - stream.foreachRDD(_ => Unit) - ssc.start() - - eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { - // Verify the offset for each group/topic to see whether they are equal to the expected one. - topics.foreach { case (t, _) => assert(getCommitOffset(t, 0) === Some(29L)) } - } - } - - /** Getting partition offset from Zookeeper. */ - private def getCommitOffset(topic: String, partition: Int): Option[Long] = { - try { - val tp = new TopicPartition(topic, partition) - Option(kc.getCommittedOffsets(Set(tp))(tp)) - } catch { - case e: SparkException => None - } - - } -} From 3675b24e67e6a9342605c4824c69e87753a8cb40 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Fri, 8 Jan 2016 18:39:38 +0200 Subject: [PATCH 13/16] Update KafkaUtils --- .../streaming/kafka/v09/KafkaUtils.scala | 220 +++++++++++++++--- 1 file changed, 186 insertions(+), 34 deletions(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala index 0dd856b01d28..00bafa2495b8 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaUtils.scala @@ -19,7 +19,10 @@ package org.apache.spark.streaming.kafka.v09 import java.io.OutputStream import java.lang.{Integer => JInt, Long => JLong} -import java.util.{Map => JMap, Set => JSet} + +import java.util.{List => JList} +import java.util.{Map => JMap} +import java.util.{Set => JSet} import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.TopicPartition @@ -40,10 +43,9 @@ import scala.reflect._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java._ -import org.apache.spark.streaming.dstream.{InputDStream, ReceiverInputDStream} +import org.apache.spark.streaming.dstream.{DStream, InputDStream} object KafkaUtils { @@ -74,17 +76,30 @@ object KafkaUtils { /** Make sure offsets are available in kafka, or throw an exception */ private def checkOffsets( - kc: KafkaCluster[_, _], - offsetRanges: Array[OffsetRange]): Unit = { - val topics = offsetRanges.map(_.topicPartition).toSet - val low = kc.getEarliestOffsets(topics) - val high = kc.getLatestOffsets(topics) - val badRanges = offsetRanges.filterNot { o => - low(o.topicPartition()) <= o.fromOffset && - o.untilOffset <= high(o.topicPartition()) + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange]): Array[OffsetRange] = { + val kc = new KafkaCluster(kafkaParams) + try { + val topics = offsetRanges.map(_.topicPartition).toSet + val low = kc.getEarliestOffsets(topics) + val high = kc.getLatestOffsetsWithLeaders(topics) + + val result = offsetRanges.filterNot { o => + low(o.topicPartition()) <= o.fromOffset && + o.untilOffset <= high(o.topicPartition()).offset + } + + if (!result.isEmpty) { + throw new SparkException("Offsets not available in Kafka: " + result.mkString(",")) + } + + offsetRanges.map { o => + OffsetRange(o.topic, o.partition, o.fromOffset, o.untilOffset, + high(o.topicPartition()).host) + } } - if (!badRanges.isEmpty) { - throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) + finally { + kc.close() } } @@ -95,9 +110,12 @@ object KafkaUtils { offsetRanges: Array[OffsetRange] ): RDD[(K, V)] = sc.withScope { val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) - val kc = new KafkaCluster[K, V](addSSLOptions(kafkaParams, sc)) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, (K, V)](sc, addSSLOptions(kafkaParams, sc), offsetRanges, messageHandler) + new KafkaRDD[K, V, (K, V)]( + sc, + addSSLOptions(kafkaParams, sc), + checkOffsets(kafkaParams, offsetRanges), + messageHandler + ) } /** @@ -122,8 +140,11 @@ object KafkaUtils { ): RDD[R] = sc.withScope { val kc = new KafkaCluster[K, V](addSSLOptions(kafkaParams, sc)) val cleanedHandler = sc.clean(messageHandler) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, R](sc, addSSLOptions(kafkaParams, sc), offsetRanges, cleanedHandler) + new KafkaRDD[K, V, R](sc, + addSSLOptions(kafkaParams, sc), + checkOffsets(kafkaParams, offsetRanges), + cleanedHandler + ) } /** @@ -254,16 +275,7 @@ object KafkaUtils { topics: Set[String] ): InputDStream[(K, V)] = { val messageHandler = (cr: ConsumerRecord[K, V]) => (cr.key, cr.value) - val kc = new KafkaCluster[K, V](addSSLOptions(kafkaParams, ssc.sparkContext)) - val reset = kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).map(_.toLowerCase) - - val fromOffsets = if (reset == Some("earliest")) { - kc.getEarliestOffsets(kc.getPartitions(topics)) - } else { - kc.getLatestOffsets(kc.getPartitions(topics)) - } - - kc.close() + val fromOffsets = getFromOffsets(kafkaParams, topics) new DirectKafkaInputDStream[K, V, (K, V)]( ssc, addSSLOptions(kafkaParams, ssc.sparkContext), fromOffsets, messageHandler) @@ -380,6 +392,147 @@ object KafkaUtils { def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = TopicAndPartition(topic, partition) + + private[kafka] def getFromOffsets( + kafkaParams: Map[String, String], + topics: Set[String] + ): Map[TopicPartition, Long] = { + val kc = new KafkaCluster(kafkaParams) + try { + val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase) + if (reset == Some("earliest")) { + kc.getEarliestOffsets(kc.getPartitions(topics)) + } else { + kc.getLatestOffsets(kc.getPartitions(topics)) + } + } + finally { + kc.close() + } + } +} + +/** + * This is a helper class that wraps the KafkaUtils.createStream() into more + * Python-friendly class and function so that it can be easily + * instantiated and called from Python's KafkaUtils (see SPARK-6027). + * + * The zero-arg constructor helps instantiate this class from the Class object + * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream() + * takes care of known parameters instead of passing them from Python + */ +private[kafka] class KafkaUtilsPythonHelper { + import KafkaUtilsPythonHelper._ + + def createRDDWithoutMessageHandler( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange]): JavaRDD[(Array[Byte], Array[Byte])] = { + val messageHandler = + (cr: ConsumerRecord[Array[Byte], Array[Byte]]) => (cr.key, cr.value) + new JavaRDD(createRDD(jsc, kafkaParams, offsetRanges, messageHandler)) + } + + def createRDDWithMessageHandler( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange]): JavaRDD[Array[Byte]] = { + val messageHandler = (cr: ConsumerRecord[Array[Byte], Array[Byte]]) => + new PythonConsumerRecord( + cr.topic, cr.partition, cr.offset, cr.key(), cr.value()) + val rdd = createRDD(jsc, kafkaParams, offsetRanges, messageHandler). + mapPartitions(picklerIterator) + new JavaRDD(rdd) + } + + private def createRDD[V: ClassTag]( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + messageHandler: ConsumerRecord[Array[Byte], Array[Byte]] => V): RDD[V] = { + kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "ByteArrayDeserializer" ) + kafkaParams.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "ByteArrayDeserializer") + KafkaUtils.createRDD[Array[Byte], Array[Byte], V]( + jsc.sc, + kafkaParams.asScala.toMap, + offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), + messageHandler + ) + } + + def createDirectStreamWithoutMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicPartition, JLong]): JavaDStream[(Array[Byte], Array[Byte])] = { + val messageHandler = + (cr: ConsumerRecord[Array[Byte], Array[Byte]]) => (cr.key, cr.value) + new JavaDStream(createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler)) + } + + def createDirectStreamWithMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicPartition, JLong]): JavaDStream[Array[Byte]] = { + val messageHandler = (cr: ConsumerRecord[Array[Byte], Array[Byte]]) => + new PythonConsumerRecord(cr.topic, cr.partition, cr.offset, cr.key(), cr.value()) + val stream = createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler). + mapPartitions(picklerIterator) + new JavaDStream(stream) + } + + private def createDirectStream[V: ClassTag]( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicPartition, JLong], + messageHandler: ConsumerRecord[Array[Byte], Array[Byte]] => V): DStream[V] = { + + val currentFromOffsets = if (!fromOffsets.isEmpty) { + val topicsFromOffsets = fromOffsets.keySet().asScala.map(_.topic) + if (topicsFromOffsets != topics.asScala.toSet) { + throw new IllegalStateException( + s"The specified topics: ${topics.asScala.toSet.mkString(" ")} " + + s"do not equal to the topic from offsets: ${topicsFromOffsets.mkString(" ")}") + } + Map(fromOffsets.asScala.mapValues { _.longValue() }.toSeq: _*) + } else { + KafkaUtils.getFromOffsets( + Map(kafkaParams.asScala.toSeq: _*), Set(topics.asScala.toSeq: _*)) + } + + kafkaParams.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, "ByteArrayDeserializer" ) + kafkaParams.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "ByteArrayDeserializer") + KafkaUtils.createDirectStream[Array[Byte], Array[Byte], V]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Map(currentFromOffsets.toSeq: _*), + messageHandler) + } + + def createOffsetRange( + topic: String, + partition: JInt, + fromOffset: JLong, + untilOffset: JLong + ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) + + def createTopicAndPartition(topic: String, partition: JInt): TopicPartition = + new TopicPartition(topic, partition) + + def offsetRangesOfKafkaRDD(rdd: RDD[_]): JList[OffsetRange] = { + val parentRDDs = rdd.getNarrowAncestors + val kafkaRDDs = parentRDDs.filter(rdd => rdd.isInstanceOf[KafkaRDD[_, _, _]]) + + require( + kafkaRDDs.length == 1, + "Cannot get offset ranges, as there may be multiple Kafka RDDs or no Kafka RDD associated" + + "with this RDD, please call this method only on a Kafka RDD.") + + val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _]] + kafkaRDD.offsetRanges.toSeq.asJava + } } private object KafkaUtilsPythonHelper { @@ -389,7 +542,7 @@ private object KafkaUtilsPythonHelper { SerDeUtil.initialize() synchronized { if (!initialized) { - new PythonMessageAndMetadataPickler().register() + new PythonConsumerRecordPickler().register() initialized = true } } @@ -401,18 +554,18 @@ private object KafkaUtilsPythonHelper { new SerDeUtil.AutoBatchedPickler(iter) } - case class PythonMessageAndMetadata( + case class PythonConsumerRecord( topic: String, partition: JInt, offset: JLong, key: Array[Byte], message: Array[Byte]) - class PythonMessageAndMetadataPickler extends IObjectPickler { + class PythonConsumerRecordPickler extends IObjectPickler { private val module = "pyspark.streaming.kafka" def register(): Unit = { - Pickler.registerCustomPickler(classOf[PythonMessageAndMetadata], this) + Pickler.registerCustomPickler(classOf[PythonConsumerRecord], this) Pickler.registerCustomPickler(this.getClass, this) } @@ -422,7 +575,7 @@ private object KafkaUtilsPythonHelper { out.write(s"$module\nKafkaMessageAndMetadata\n".getBytes(UTF_8)) } else { pickler.save(this) - val msgAndMetaData = obj.asInstanceOf[PythonMessageAndMetadata] + val msgAndMetaData = obj.asInstanceOf[PythonConsumerRecord] out.write(Opcodes.MARK) pickler.save(msgAndMetaData.topic) pickler.save(msgAndMetaData.partition) @@ -434,5 +587,4 @@ private object KafkaUtilsPythonHelper { } } } - } From 1b8f23a6314a636fc1c57a703b6060bb1e390b6c Mon Sep 17 00:00:00 2001 From: nikit-os Date: Fri, 8 Jan 2016 18:40:44 +0200 Subject: [PATCH 14/16] Remove 09KafkaWordCount example for receiver-based InputStream --- .../examples/streaming/KafkaWordCount.scala | 41 ------------------- 1 file changed, 41 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala index 6115fd28f073..b40d17e9c2fa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/KafkaWordCount.scala @@ -65,47 +65,6 @@ object KafkaWordCount { } } -/** - * Consumes messages from one or more topics in Kafka and does wordcount. - * Usage: v09KafkaWordCount - * is a list of one or more brokers servers - * is the name of kafka consumer group - * is a list of one or more kafka topics to consume from - * is the number of threads the kafka consumer should use - * - * Example: - * `$ bin/run-example \ - * org.apache.spark.examples.streaming.v09KafkaWordCount broker01,broker02,broker03 \ - * my-consumer-group topic1,topic2 1` - */ -object v09KafkaWordCount { - def main(args: Array[String]) { - import org.apache.spark.streaming.kafka.v09._ - - if (args.length < 4) { - System.err.println("Usage: v09KafkaWordCount ") - System.exit(1) - } - - StreamingExamples.setStreamingLogLevels() - - val Array(brokers, group, topics, numThreads) = args - val sparkConf = new SparkConf().setAppName("v09KafkaWordCount") - val ssc = new StreamingContext(sparkConf, Seconds(2)) - ssc.checkpoint("checkpoint") - - val topicMap = topics.split(",").map((_, numThreads.toInt)).toMap - val lines = KafkaUtils.createStream(ssc, brokers, group, topicMap).map(_._2) - val words = lines.flatMap(_.split(" ")) - val wordCounts = words.map(x => (x, 1L)) - .reduceByKeyAndWindow(_ + _, _ - _, Minutes(10), Seconds(2), 2) - wordCounts.print() - - ssc.start() - ssc.awaitTermination() - } -} - // Produces some random words between 1 and 100. object KafkaWordCountProducer { From 8312792f488aea4507c757d1c3810beb4ef16575 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Fri, 8 Jan 2016 18:54:32 +0200 Subject: [PATCH 15/16] Fix kc.getLatestOffsets get() in DirectKafkaStreamSuite --- .../spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala index 0b181685dfc9..cb4dab6a2cf4 100644 --- a/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala +++ b/external/kafka-v09/src/test/scala/org/apache/spark/streaming/kafka/v09/DirectKafkaStreamSuite.scala @@ -157,7 +157,7 @@ class DirectKafkaStreamSuite "spark.kafka.poll.time" -> "100") val kc = new KafkaCluster(kafkaParams) def getLatestOffset(): Long = { - kc.getLatestOffsets(Set(topicPartition)).get(topicPartition).get + kc.getLatestOffsets(Set(topicPartition)).get(topicPartition).getOrElse(0) } // Send some initial messages before starting context @@ -205,7 +205,7 @@ class DirectKafkaStreamSuite "spark.kafka.poll.time" -> "100") val kc = new KafkaCluster(kafkaParams) def getLatestOffset(): Long = { - kc.getLatestOffsets(Set(topicPartition)).get(topicPartition).get + kc.getLatestOffsets(Set(topicPartition)).get(topicPartition).getOrElse(0) } // Send some initial messages before starting context From 0fcda2881b196265239fd38044b23f7ddcb53b45 Mon Sep 17 00:00:00 2001 From: nikit-os Date: Fri, 8 Jan 2016 19:16:55 +0200 Subject: [PATCH 16/16] implement consumer.poll(x) correctly based on https://issues.apache.org/jira/browse/KAFKA-3044 --- .../spark/streaming/kafka/v09/KafkaRDD.scala | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala index 95c7b4922368..debbc9c3352e 100644 --- a/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala +++ b/external/kafka-v09/src/main/scala/org/apache/spark/streaming/kafka/v09/KafkaRDD.scala @@ -29,6 +29,7 @@ import org.apache.spark.{ Logging, Partition, SparkContext, TaskContext } import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag +import scala.collection.JavaConverters._ /** * A batch-oriented interface for consuming from Kafka. @@ -154,7 +155,7 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( consumer.assign(Collections.singletonList[TopicPartition](tp)) var requestOffset = part.fromOffset - var iter: java.util.Iterator[ConsumerRecord[K, V]] = null + var iter: Iterator[ConsumerRecord[K, V]] = null consumer.seek(tp, requestOffset) override def close(): Unit = { @@ -163,12 +164,26 @@ class KafkaRDD[K: ClassTag, V: ClassTag, R: ClassTag] private[spark] ( } } + private def fetchBatch: Iterator[ConsumerRecord[K, V]] = { + consumer.seek(new TopicPartition(part.topic, part.partition), requestOffset) + val recs = consumer.poll(pollTime) + recs.records(new TopicPartition(part.topic, part.partition)).iterator().asScala + } + override def getNext(): R = { + if ( requestOffset == part.untilOffset ) { + finished = true + null.asInstanceOf[R] + } + if (iter == null || !iter.hasNext) { - iter = consumer.poll(pollTime).iterator() + iter = fetchBatch } if (!iter.hasNext) { + if ( requestOffset < part.untilOffset ) { + return getNext() + } assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) finished = true null.asInstanceOf[R]