From aaef0fc6e7e3aae18e4e03271bc0707d09d243e4 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Thu, 3 Mar 2016 16:55:16 -0600 Subject: [PATCH 01/37] [SPARK-12177][Streaming][Kafka] separate project for starting work on beta consumer, modify getPreferredLocations to choose a consistent executor per topicpartition --- external/kafka-beta-assembly/pom.xml | 186 +++++ external/kafka-beta/pom.xml | 98 +++ .../apache/spark/streaming/kafka/Broker.scala | 66 ++ .../kafka/DirectKafkaInputDStream.scala | 214 ++++++ .../spark/streaming/kafka/KafkaCluster.scala | 425 +++++++++++ .../spark/streaming/kafka/KafkaRDD.scala | 291 ++++++++ .../streaming/kafka/KafkaRDDPartition.scala | 41 ++ .../streaming/kafka/KafkaTestUtils.scala | 269 +++++++ .../spark/streaming/kafka/KafkaUtils.scala | 659 ++++++++++++++++++ .../spark/streaming/kafka/OffsetRange.scala | 109 +++ .../spark/streaming/kafka/package-info.java | 21 + .../spark/streaming/kafka/package.scala | 23 + .../kafka/JavaDirectKafkaStreamSuite.java | 175 +++++ .../streaming/kafka/JavaKafkaRDDSuite.java | 156 +++++ .../src/test/resources/log4j.properties | 28 + .../kafka/DirectKafkaStreamSuite.scala | 470 +++++++++++++ .../streaming/kafka/KafkaClusterSuite.scala | 81 +++ .../spark/streaming/kafka/KafkaRDDSuite.scala | 175 +++++ pom.xml | 2 + project/SparkBuild.scala | 10 +- 20 files changed, 3494 insertions(+), 5 deletions(-) create mode 100644 external/kafka-beta-assembly/pom.xml create mode 100644 external/kafka-beta/pom.xml create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package-info.java create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package.scala create mode 100644 external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java create mode 100644 external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java create mode 100644 external/kafka-beta/src/test/resources/log4j.properties create mode 100644 external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala create mode 100644 external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala create mode 100644 external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala diff --git a/external/kafka-beta-assembly/pom.xml b/external/kafka-beta-assembly/pom.xml new file mode 100644 index 0000000000000..139f5d348768f --- /dev/null +++ b/external/kafka-beta-assembly/pom.xml @@ -0,0 +1,186 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-beta-assembly_2.11 + jar + Spark Project External Kafka Assembly + http://spark.apache.org/ + + + streaming-kafka-beta-assembly + + + + + org.apache.spark + spark-streaming-kafka-beta_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + provided + + + + commons-codec + commons-codec + provided + + + commons-lang + commons-lang + provided + + + com.google.protobuf + protobuf-java + provided + + + com.sun.jersey + jersey-server + provided + + + com.sun.jersey + jersey-core + provided + + + net.jpountz.lz4 + lz4 + provided + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.avro + avro-mapred + ${avro.mapred.classifier} + provided + + + org.apache.curator + curator-recipes + provided + + + org.apache.zookeeper + zookeeper + provided + + + log4j + log4j + provided + + + net.java.dev.jets3t + jets3t + provided + + + org.scala-lang + scala-library + provided + + + org.slf4j + slf4j-api + provided + + + org.slf4j + slf4j-log4j12 + provided + + + org.xerial.snappy + snappy-java + provided + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-shade-plugin + + false + + + *:* + + + + + *:* + + META-INF/*.SF + META-INF/*.DSA + META-INF/*.RSA + + + + + + + package + + shade + + + + + + reference.conf + + + log4j.properties + + + + + + + + + + + + diff --git a/external/kafka-beta/pom.xml b/external/kafka-beta/pom.xml new file mode 100644 index 0000000000000..a14b94a9aa0e5 --- /dev/null +++ b/external/kafka-beta/pom.xml @@ -0,0 +1,98 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.11 + 2.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + spark-streaming-kafka-beta_2.11 + + streaming-kafka-beta + + jar + Spark Project External Kafka + http://spark.apache.org/ + + + + 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.1 + + + 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-beta/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala new file mode 100644 index 0000000000000..9159051ba06e4 --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/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 + +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-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala new file mode 100644 index 0000000000000..54d8c8b03f206 --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -0,0 +1,214 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.annotation.tailrec +import scala.collection.mutable +import scala.reflect.ClassTag + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.Decoder + +import org.apache.spark.{Logging, SparkException} +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset +import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} +import org.apache.spark.streaming.scheduler.rate.RateEstimator + +/** + * 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 + * @param messageHandler function for translating each message into the desired type + */ +private[streaming] +class DirectKafkaInputDStream[ + K: ClassTag, + V: ClassTag, + U <: Decoder[K]: ClassTag, + T <: Decoder[V]: ClassTag, + R: ClassTag]( + _ssc: StreamingContext, + val kafkaParams: Map[String, String], + val fromOffsets: Map[TopicAndPartition, Long], + messageHandler: MessageAndMetadata[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 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, context.graph.batchDuration))) + } else { + None + } + } + + protected val kc = new KafkaCluster(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 + + @tailrec + protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { + val o = kc.getLatestLeaderOffsets(currentOffsets.keySet) + // Either.fold would confuse @tailrec, do it manually + if (o.isLeft) { + val err = o.left.get.toString + if (retries <= 0) { + throw new SparkException(err) + } else { + log.error(err) + Thread.sleep(kc.config.refreshLeaderBackoffMs) + latestLeaderOffsets(retries - 1) + } + } else { + o.right.get + } + } + + // limits the maximum number of messages per partition + protected def clamp( + leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = { + maxMessagesPerPartition.map { mmp => + leaderOffsets.map { case (tp, lo) => + tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset)) + } + }.getOrElse(leaderOffsets) + } + + override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = { + val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) + val rdd = KafkaRDD[K, V, U, T, 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.offset) + } + 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.offset) + Some(rdd) + } + + override def start(): Unit = { + } + + def stop(): Unit = { + } + + 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, U, T, 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 + val leaders = KafkaCluster.checkErrors(kc.findLeaders(topics)) + + 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, U, T, R]( + context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, 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-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala new file mode 100644 index 0000000000000..8a66621a3125c --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -0,0 +1,425 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.util.Properties + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer +import scala.util.Random +import scala.util.control.NonFatal + +import kafka.api._ +import kafka.common.{ErrorMapping, OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} +import kafka.consumer.{ConsumerConfig, SimpleConsumer} + +import org.apache.spark.SparkException +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Convenience methods for interacting with a Kafka cluster. + * See + * A Guide To The Kafka Protocol for more details on individual api calls. + * @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 + */ +@DeveloperApi +class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { + import KafkaCluster.{Err, LeaderOffset, SimpleConsumerConfig} + + // ConsumerConfig isn't serializable + @transient private var _config: SimpleConsumerConfig = null + + def config: SimpleConsumerConfig = this.synchronized { + if (_config == null) { + _config = SimpleConsumerConfig(kafkaParams) + } + _config + } + + def connect(host: String, port: Int): SimpleConsumer = + new SimpleConsumer(host, port, config.socketTimeoutMs, + config.socketReceiveBufferBytes, config.clientId) + + def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] = + findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2)) + + // Metadata api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI + // scalastyle:on + + def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = { + val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, + 0, config.clientId, Seq(topic)) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp: TopicMetadataResponse = consumer.send(req) + resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata => + tm.partitionsMetadata.find(_.partitionId == partition) + }.foreach { pm: PartitionMetadata => + pm.leader.foreach { leader => + return Right((leader.host, leader.port)) + } + } + } + Left(errs) + } + + def findLeaders( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, (String, Int)]] = { + val topics = topicAndPartitions.map(_.topic) + val response = getPartitionMetadata(topics).right + val answer = response.flatMap { tms: Set[TopicMetadata] => + val leaderMap = tms.flatMap { tm: TopicMetadata => + tm.partitionsMetadata.flatMap { pm: PartitionMetadata => + val tp = TopicAndPartition(tm.topic, pm.partitionId) + if (topicAndPartitions(tp)) { + pm.leader.map { l => + tp -> (l.host -> l.port) + } + } else { + None + } + } + }.toMap + + if (leaderMap.keys.size == topicAndPartitions.size) { + Right(leaderMap) + } else { + val missing = topicAndPartitions.diff(leaderMap.keySet) + val err = new Err + err.append(new SparkException(s"Couldn't find leaders for ${missing}")) + Left(err) + } + } + answer + } + + def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = { + getPartitionMetadata(topics).right.map { r => + r.flatMap { tm: TopicMetadata => + tm.partitionsMetadata.map { pm: PartitionMetadata => + TopicAndPartition(tm.topic, pm.partitionId) + } + } + } + } + + def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = { + val req = TopicMetadataRequest( + TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp: TopicMetadataResponse = consumer.send(req) + val respErrs = resp.topicsMetadata.filter(m => m.errorCode != ErrorMapping.NoError) + + if (respErrs.isEmpty) { + return Right(resp.topicsMetadata.toSet) + } else { + respErrs.foreach { m => + val cause = ErrorMapping.exceptionFor(m.errorCode) + val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" + errs.append(new SparkException(msg, cause)) + } + } + } + Left(errs) + } + + // Leader offset api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI + // scalastyle:on + + def getLatestLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = + getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime) + + def getEarliestLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = + getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime) + + def getLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition], + before: Long + ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = { + getLeaderOffsets(topicAndPartitions, before, 1).right.map { r => + r.map { kv => + // mapValues isnt serializable, see SI-7005 + kv._1 -> kv._2.head + } + } + } + + private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] = + m.groupBy(_._2).map { kv => + kv._1 -> kv._2.keys.toSeq + } + + def getLeaderOffsets( + topicAndPartitions: Set[TopicAndPartition], + before: Long, + maxNumOffsets: Int + ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = { + findLeaders(topicAndPartitions).right.flatMap { tpToLeader => + val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader) + val leaders = leaderToTp.keys + var result = Map[TopicAndPartition, Seq[LeaderOffset]]() + val errs = new Err + withBrokers(leaders, errs) { consumer => + val partitionsToGetOffsets: Seq[TopicAndPartition] = + leaderToTp((consumer.host, consumer.port)) + val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition => + tp -> PartitionOffsetRequestInfo(before, maxNumOffsets) + }.toMap + val req = OffsetRequest(reqMap) + val resp = consumer.getOffsetsBefore(req) + val respMap = resp.partitionErrorAndOffsets + partitionsToGetOffsets.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { por: PartitionOffsetsResponse => + if (por.error == ErrorMapping.NoError) { + if (por.offsets.nonEmpty) { + result += tp -> por.offsets.map { off => + LeaderOffset(consumer.host, consumer.port, off) + } + } else { + errs.append(new SparkException( + s"Empty offsets for ${tp}, is ${before} before log beginning?")) + } + } else { + errs.append(ErrorMapping.exceptionFor(por.error)) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}")) + Left(errs) + } + } + + // Consumer offset api + // scalastyle:off + // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI + // scalastyle:on + + // this 0 here indicates api version, in this case the original ZK backed api. + private def defaultConsumerApiVersion: Short = 0 + + /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ + def getConsumerOffsets( + groupId: String, + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, Long]] = + getConsumerOffsets(groupId, topicAndPartitions, defaultConsumerApiVersion) + + def getConsumerOffsets( + groupId: String, + topicAndPartitions: Set[TopicAndPartition], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, Long]] = { + getConsumerOffsetMetadata(groupId, topicAndPartitions, consumerApiVersion).right.map { r => + r.map { kv => + kv._1 -> kv._2.offset + } + } + } + + /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ + def getConsumerOffsetMetadata( + groupId: String, + topicAndPartitions: Set[TopicAndPartition] + ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = + getConsumerOffsetMetadata(groupId, topicAndPartitions, defaultConsumerApiVersion) + + def getConsumerOffsetMetadata( + groupId: String, + topicAndPartitions: Set[TopicAndPartition], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = { + var result = Map[TopicAndPartition, OffsetMetadataAndError]() + val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq, consumerApiVersion) + val errs = new Err + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp = consumer.fetchOffsets(req) + val respMap = resp.requestInfo + val needed = topicAndPartitions.diff(result.keySet) + needed.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { ome: OffsetMetadataAndError => + if (ome.error == ErrorMapping.NoError) { + result += tp -> ome + } else { + errs.append(ErrorMapping.exceptionFor(ome.error)) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}")) + Left(errs) + } + + /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ + def setConsumerOffsets( + groupId: String, + offsets: Map[TopicAndPartition, Long] + ): Either[Err, Map[TopicAndPartition, Short]] = + setConsumerOffsets(groupId, offsets, defaultConsumerApiVersion) + + def setConsumerOffsets( + groupId: String, + offsets: Map[TopicAndPartition, Long], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, Short]] = { + val meta = offsets.map { kv => + kv._1 -> OffsetAndMetadata(kv._2) + } + setConsumerOffsetMetadata(groupId, meta, consumerApiVersion) + } + + /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ + def setConsumerOffsetMetadata( + groupId: String, + metadata: Map[TopicAndPartition, OffsetAndMetadata] + ): Either[Err, Map[TopicAndPartition, Short]] = + setConsumerOffsetMetadata(groupId, metadata, defaultConsumerApiVersion) + + def setConsumerOffsetMetadata( + groupId: String, + metadata: Map[TopicAndPartition, OffsetAndMetadata], + consumerApiVersion: Short + ): Either[Err, Map[TopicAndPartition, Short]] = { + var result = Map[TopicAndPartition, Short]() + val req = OffsetCommitRequest(groupId, metadata, consumerApiVersion) + val errs = new Err + val topicAndPartitions = metadata.keySet + withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => + val resp = consumer.commitOffsets(req) + val respMap = resp.commitStatus + val needed = topicAndPartitions.diff(result.keySet) + needed.foreach { tp: TopicAndPartition => + respMap.get(tp).foreach { err: Short => + if (err == ErrorMapping.NoError) { + result += tp -> err + } else { + errs.append(ErrorMapping.exceptionFor(err)) + } + } + } + if (result.keys.size == topicAndPartitions.size) { + return Right(result) + } + } + val missing = topicAndPartitions.diff(result.keySet) + errs.append(new SparkException(s"Couldn't set offsets for ${missing}")) + Left(errs) + } + + // Try a call against potentially multiple brokers, accumulating errors + private def withBrokers(brokers: Iterable[(String, Int)], errs: Err) + (fn: SimpleConsumer => Any): Unit = { + brokers.foreach { hp => + var consumer: SimpleConsumer = null + try { + consumer = connect(hp._1, hp._2) + fn(consumer) + } catch { + case NonFatal(e) => + errs.append(e) + } finally { + if (consumer != null) { + consumer.close() + } + } + } + } +} + +@DeveloperApi +object KafkaCluster { + 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 + ) + } + + case class LeaderOffset(host: String, port: Int, offset: Long) + + /** + * High-level kafka consumers connect to ZK. ConsumerConfig assumes this use case. + * Simple consumers connect directly to brokers, but need many of the same configs. + * This subclass won't warn about missing ZK params, or presence of broker params. + */ + class SimpleConsumerConfig private(brokers: String, originalProps: Properties) + extends ConsumerConfig(originalProps) { + val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => + val hpa = hp.split(":") + if (hpa.size == 1) { + throw new SparkException(s"Broker not in the correct format of : [$brokers]") + } + (hpa(0), hpa(1).toInt) + } + } + + object SimpleConsumerConfig { + /** + * Make a consumer config without requiring group.id or zookeeper.connect, + * since communicating with brokers also needs common settings such as timeout + */ + def apply(kafkaParams: Map[String, String]): SimpleConsumerConfig = { + // These keys are from other pre-existing kafka configs for specifying brokers, accept either + val brokers = kafkaParams.get("metadata.broker.list") + .orElse(kafkaParams.get("bootstrap.servers")) + .getOrElse(throw new SparkException( + "Must specify metadata.broker.list or bootstrap.servers")) + + val props = new Properties() + kafkaParams.foreach { case (key, value) => + // prevent warnings on parameters ConsumerConfig doesn't know about + if (key != "metadata.broker.list" && key != "bootstrap.servers") { + props.put(key, value) + } + } + + Seq("zookeeper.connect", "group.id").foreach { s => + if (!props.containsKey(s)) { + props.setProperty(s, "") + } + } + + new SimpleConsumerConfig(brokers, props) + } + } +} diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala new file mode 100644 index 0000000000000..5e9ef2eadf90e --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -0,0 +1,291 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.{classTag, ClassTag} + +import kafka.api.{FetchRequestBuilder, FetchResponse} +import kafka.common.{ErrorMapping, TopicAndPartition} +import kafka.consumer.SimpleConsumer +import kafka.message.{MessageAndMetadata, MessageAndOffset} +import kafka.serializer.Decoder +import kafka.utils.VerifiableProperties + +import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.util.NextIterator + +/** + * 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 + * @param messageHandler function for translating each message into the desired type + */ + +class KafkaRDD[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag] private[spark] ( + sc: SparkContext, + kafkaParams: Map[String, String], + val offsetRanges: Array[OffsetRange], + leaders: Map[TopicAndPartition, (String, Int)], + messageHandler: MessageAndMetadata[K, V] => R + ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { + override def getPartitions: Array[Partition] = { + offsetRanges.zipWithIndex.map { case (o, i) => + val (host, port) = leaders(TopicAndPartition(o.topic, o.partition)) + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) + }.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 + } + + // TODO is there a better way to get executors + @transient private var sortedExecutors: Array[ExecutorCacheTaskLocation] = null + private def executors(): Array[ExecutorCacheTaskLocation] = { + if (null == sortedExecutors) { + val bm = sparkContext.env.blockManager + sortedExecutors = bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith((a, b) => a.host > b.host || a.executorId > b.executorId) + } + sortedExecutors + } + + // non-negative modulus, from java 8 math + private def floorMod(a: Int, b: Int): Int = ((a % b) + b) % b + + override def getPreferredLocations(thePart: Partition): Seq[String] = { + // want lead broker if an executor is running on it, otherwise stable exec to use caching + // TODO what if broker host is ip and executor is name, or vice versa + val part = thePart.asInstanceOf[KafkaRDDPartition] + val allExecs = executors() + val brokerExecs = allExecs.filter(_.host == part.host) + val execs = if (brokerExecs.isEmpty) allExecs else brokerExecs + val index = floorMod(part.topic.hashCode * 41 + part.partition.hashCode, execs.length) + val chosen = execs(index) + + Seq(chosen.toString) + } + + 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 kc = new KafkaCluster(kafkaParams) + val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[K]] + val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) + .newInstance(kc.config.props) + .asInstanceOf[Decoder[V]] + val consumer = connectLeader + var requestOffset = part.fromOffset + var iter: Iterator[MessageAndOffset] = null + + // The idea is to use the provided preferred host, except on task retry attempts, + // to minimize number of kafka metadata requests + private def connectLeader: SimpleConsumer = { + if (context.attemptNumber > 0) { + kc.connectLeader(part.topic, part.partition).fold( + errs => throw new SparkException( + s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " + + errs.mkString("\n")), + consumer => consumer + ) + } else { + kc.connect(part.host, part.port) + } + } + + private def handleFetchErr(resp: FetchResponse) { + if (resp.hasError) { + val err = resp.errorCode(part.topic, part.partition) + if (err == ErrorMapping.LeaderNotAvailableCode || + err == ErrorMapping.NotLeaderForPartitionCode) { + log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " + + s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms") + Thread.sleep(kc.config.refreshLeaderBackoffMs) + } + // Let normal rdd retry sort out reconnect attempts + throw ErrorMapping.exceptionFor(err) + } + } + + private def fetchBatch: Iterator[MessageAndOffset] = { + val req = new FetchRequestBuilder() + .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes) + .build() + val resp = consumer.fetch(req) + handleFetchErr(resp) + // kafka may return a batch that starts before the requested offset + resp.messageSet(part.topic, part.partition) + .iterator + .dropWhile(_.offset < requestOffset) + } + + override def close(): Unit = { + if (consumer != null) { + consumer.close() + } + } + + override def getNext(): R = { + if (iter == null || !iter.hasNext) { + iter = fetchBatch + } + if (!iter.hasNext) { + assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) + finished = true + null.asInstanceOf[R] + } else { + val item = iter.next() + if (item.offset >= part.untilOffset) { + assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) + finished = true + null.asInstanceOf[R] + } else { + requestOffset = item.nextOffset + messageHandler(new MessageAndMetadata( + part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder)) + } + } + } + } +} + +private[kafka] +object KafkaRDD { + import KafkaCluster.LeaderOffset + + /** + * @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 + * @param messageHandler function for translating each message into the desired type + */ + def apply[ + K: ClassTag, + V: ClassTag, + U <: Decoder[_]: ClassTag, + T <: Decoder[_]: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + untilOffsets: Map[TopicAndPartition, LeaderOffset], + messageHandler: MessageAndMetadata[K, V] => R + ): KafkaRDD[K, V, U, T, R] = { + val leaders = untilOffsets.map { case (tp, lo) => + tp -> (lo.host, lo.port) + }.toMap + + val offsetRanges = fromOffsets.map { case (tp, fo) => + val uo = untilOffsets(tp) + OffsetRange(tp.topic, tp.partition, fo, uo.offset) + }.toArray + + new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) + } +} diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala new file mode 100644 index 0000000000000..a660d2a00c35d --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import 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 host preferred kafka host, i.e. the leader at the time the rdd was created + * @param port preferred kafka host's port + */ +private[kafka] +class KafkaRDDPartition( + val index: Int, + val topic: String, + val partition: Int, + val fromOffset: Long, + val untilOffset: Long, + val host: String, + val port: Int +) extends Partition { + /** Number of messages this partition refers to */ + def count(): Long = untilOffset - fromOffset +} diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala new file mode 100644 index 0000000000000..2bc27b88ed7a1 --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap, Properties} +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.language.postfixOps +import scala.util.control.NonFatal + +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.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils + +/** + * 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 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: ZkUtils = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkUtils).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 + zkUtils = ZkUtils(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, false) + 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 (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-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala new file mode 100644 index 0000000000000..c94b1855e2821 --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -0,0 +1,659 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.OutputStream +import java.lang.{Integer => JInt, Long => JLong} +import java.util.{List => JList, Map => JMap, Set => JSet} + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import com.google.common.base.Charsets.UTF_8 +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} +import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} + +import org.apache.spark.{SparkContext, SparkException} +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.api.java.function.{Function => JFunction} +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} +import org.apache.spark.streaming.util.WriteAheadLogUtils + +object KafkaUtils { + /** get leaders for the given offset ranges, or throw an exception */ + private def leadersForRanges( + kc: KafkaCluster, + offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { + val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet + val leaders = kc.findLeaders(topics) + KafkaCluster.checkErrors(leaders) + } + + /** 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.getEarliestLeaderOffsets(topics).right + high <- kc.getLatestLeaderOffsets(topics).right + } yield { + offsetRanges.filterNot { o => + low(o.topicAndPartition).offset <= o.fromOffset && + o.untilOffset <= high(o.topicAndPartition).offset + } + } + val badRanges = KafkaCluster.checkErrors(result) + if (!badRanges.isEmpty) { + throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) + } + } + + private[kafka] def getFromOffsets( + kc: KafkaCluster, + kafkaParams: Map[String, String], + topics: Set[String] + ): Map[TopicAndPartition, Long] = { + val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase) + val result = for { + topicPartitions <- kc.getPartitions(topics).right + leaderOffsets <- (if (reset == Some("smallest")) { + kc.getEarliestLeaderOffsets(topicPartitions) + } else { + kc.getLatestLeaderOffsets(topicPartitions) + }).right + } yield { + leaderOffsets.map { case (tp, lo) => + (tp, lo.offset) + } + } + KafkaCluster.checkErrors(result) + } + + /** + * Create a RDD from Kafka using offset ranges for each topic and partition. + * + * @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 + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return RDD of (Kafka message key, Kafka message value) + */ + def createRDD[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange] + ): RDD[(K, V)] = sc.withScope { + val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) + val kc = new KafkaCluster(kafkaParams) + val leaders = leadersForRanges(kc, offsetRanges) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, 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 leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, + * in which case leaders will be looked up on the driver. + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return RDD of R + */ + def createRDD[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, + R: ClassTag]( + sc: SparkContext, + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange], + leaders: Map[TopicAndPartition, Broker], + messageHandler: MessageAndMetadata[K, V] => R + ): RDD[R] = sc.withScope { + val kc = new KafkaCluster(kafkaParams) + val leaderMap = if (leaders.isEmpty) { + leadersForRanges(kc, offsetRanges) + } else { + // This could be avoided by refactoring KafkaRDD.leaders and KafkaCluster to use Broker + leaders.map { + case (tp: TopicAndPartition, Broker(host, port)) => (tp, (host, port)) + } + } + val cleanedHandler = sc.clean(messageHandler) + checkOffsets(kc, offsetRanges) + new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, cleanedHandler) + } + + /** + * Create a RDD from Kafka using offset ranges for each topic and partition. + * + * @param jsc JavaSparkContext 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 keyClass type of Kafka message key + * @param valueClass type of Kafka message value + * @param keyDecoderClass type of Kafka message key decoder + * @param valueDecoderClass type of Kafka message value decoder + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return RDD of (Kafka message key, Kafka message value) + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + 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) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + new JavaPairRDD(createRDD[K, V, KD, VD]( + 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 "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 leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, + * in which case leaders will be looked up on the driver. + * @param messageHandler Function for translating each message and metadata into the desired type + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return RDD of R + */ + def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + offsetRanges: Array[OffsetRange], + leaders: JMap[TopicAndPartition, Broker], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaRDD[R] = jsc.sc.withScope { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val leaderMap = Map(leaders.asScala.toSeq: _*) + createRDD[K, V, KD, VD, R]( + jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, leaderMap, 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.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 + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return DStream of R + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag, + R: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + fromOffsets: Map[TopicAndPartition, Long], + messageHandler: MessageAndMetadata[K, V] => R + ): InputDStream[R] = { + val cleanedHandler = ssc.sc.clean(messageHandler) + new DirectKafkaInputDStream[K, V, KD, VD, R]( + ssc, kafkaParams, fromOffsets, 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.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 + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createDirectStream[ + K: ClassTag, + V: ClassTag, + KD <: Decoder[K]: ClassTag, + VD <: Decoder[V]: ClassTag] ( + ssc: StreamingContext, + kafkaParams: Map[String, String], + topics: Set[String] + ): InputDStream[(K, V)] = { + val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) + val kc = new KafkaCluster(kafkaParams) + val fromOffsets = getFromOffsets(kc, kafkaParams, topics) + new DirectKafkaInputDStream[K, V, KD, VD, (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.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 keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class of the value decoder + * @param recordClass Class of the records in DStream + * @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 + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @tparam R type returned by messageHandler + * @return DStream of R + */ + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + recordClass: Class[R], + kafkaParams: JMap[String, String], + fromOffsets: JMap[TopicAndPartition, JLong], + messageHandler: JFunction[MessageAndMetadata[K, V], R] + ): JavaInputDStream[R] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) + val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) + createDirectStream[K, V, KD, VD, 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.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 keyDecoderClass Class of the key decoder + * @param valueDecoderClass Class type of the value decoder + * @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 + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + * @tparam KD type of Kafka message key decoder + * @tparam VD type of Kafka message value decoder + * @return DStream of (Kafka message key, Kafka message value) + */ + def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + keyDecoderClass: Class[KD], + valueDecoderClass: Class[VD], + kafkaParams: JMap[String, String], + topics: JSet[String] + ): JavaPairInputDStream[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) + implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) + createDirectStream[K, V, KD, VD]( + jssc.ssc, + Map(kafkaParams.asScala.toSeq: _*), + Set(topics.asScala.toSeq: _*) + ) + } +} + +/** + * 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], + leaders: JMap[TopicAndPartition, Broker]): JavaRDD[(Array[Byte], Array[Byte])] = { + val messageHandler = + (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) + new JavaRDD(createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler)) + } + + def createRDDWithMessageHandler( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker]): JavaRDD[Array[Byte]] = { + val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => + new PythonMessageAndMetadata( + mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) + val rdd = createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler). + mapPartitions(picklerIterator) + new JavaRDD(rdd) + } + + private def createRDD[V: ClassTag]( + jsc: JavaSparkContext, + kafkaParams: JMap[String, String], + offsetRanges: JList[OffsetRange], + leaders: JMap[TopicAndPartition, Broker], + messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): RDD[V] = { + KafkaUtils.createRDD[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( + jsc.sc, + kafkaParams.asScala.toMap, + offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), + leaders.asScala.toMap, + messageHandler + ) + } + + def createDirectStreamWithoutMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JLong]): JavaDStream[(Array[Byte], Array[Byte])] = { + val messageHandler = + (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) + new JavaDStream(createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler)) + } + + def createDirectStreamWithMessageHandler( + jssc: JavaStreamingContext, + kafkaParams: JMap[String, String], + topics: JSet[String], + fromOffsets: JMap[TopicAndPartition, JLong]): JavaDStream[Array[Byte]] = { + val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => + new PythonMessageAndMetadata(mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) + 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[TopicAndPartition, JLong], + messageHandler: MessageAndMetadata[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 { + val kc = new KafkaCluster(Map(kafkaParams.asScala.toSeq: _*)) + KafkaUtils.getFromOffsets( + kc, Map(kafkaParams.asScala.toSeq: _*), Set(topics.asScala.toSeq: _*)) + } + + KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, 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): TopicAndPartition = + TopicAndPartition(topic, partition) + + def createBroker(host: String, port: JInt): Broker = Broker(host, port) + + 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 { + 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-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala new file mode 100644 index 0000000000000..d9b856e4697a0 --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/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 + +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-beta/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package-info.java new file mode 100644 index 0000000000000..947bae115a620 --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package-info.java @@ -0,0 +1,21 @@ +/* + * 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. + */ + +/** + * Kafka receiver for spark streaming. + */ +package org.apache.spark.streaming.kafka; \ No newline at end of file diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package.scala new file mode 100644 index 0000000000000..47c5187f8751f --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package.scala @@ -0,0 +1,23 @@ +/* + * 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 receiver for spark streaming, + */ +package object kafka diff --git a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java new file mode 100644 index 0000000000000..4891e4f4a17bc --- /dev/null +++ b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.*; +import java.util.concurrent.atomic.AtomicReference; + +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.JavaPairRDD; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; +import org.apache.spark.streaming.Durations; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaStreamingContext; + +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 = "topic1"; + final String topic2 = "topic2"; + // 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); + + Set sent = new HashSet<>(); + sent.addAll(Arrays.asList(topic1data)); + sent.addAll(Arrays.asList(topic2data)); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + kafkaParams.put("auto.offset.reset", "smallest"); + + JavaDStream stream1 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + topicToSet(topic1) + ).transformToPair( + // Make sure you can get offset ranges from the rdd + new Function, JavaPairRDD>() { + @Override + public JavaPairRDD call(JavaPairRDD rdd) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic1, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) { + return kv._2(); + } + } + ); + + JavaDStream stream2 = KafkaUtils.createDirectStream( + ssc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + topicOffsetToMap(topic2, 0L), + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) { + return msgAndMd.message(); + } + } + ); + JavaDStream unifiedStream = stream1.union(stream2); + + final Set result = Collections.synchronizedSet(new HashSet()); + unifiedStream.foreachRDD(new VoidFunction>() { + @Override + 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() + ); + } + } + } + ); + 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 static Set topicToSet(String topic) { + Set topicSet = new HashSet<>(); + topicSet.add(topic); + return topicSet; + } + + private static Map topicOffsetToMap(String topic, Long offsetToStart) { + Map 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-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java new file mode 100644 index 0000000000000..afcc6cfccd39a --- /dev/null +++ b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/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; + +import java.io.Serializable; +import java.util.HashMap; +import java.util.Map; + +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 = "topic1"; + String topic2 = "topic2"; + + createTopicAndSendData(topic1); + createTopicAndSendData(topic2); + + Map kafkaParams = new HashMap<>(); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + + OffsetRange[] offsetRanges = { + OffsetRange.create(topic1, 0, 0, 1), + OffsetRange.create(topic2, 0, 0, 1) + }; + + Map emptyLeaders = new HashMap<>(); + Map 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, + StringDecoder.class, + StringDecoder.class, + kafkaParams, + offsetRanges + ).map( + new Function, String>() { + @Override + public String call(Tuple2 kv) { + return kv._2(); + } + } + ); + + JavaRDD rdd2 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + offsetRanges, + emptyLeaders, + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) { + return msgAndMd.message(); + } + } + ); + + JavaRDD rdd3 = KafkaUtils.createRDD( + sc, + String.class, + String.class, + StringDecoder.class, + StringDecoder.class, + String.class, + kafkaParams, + offsetRanges, + leaders, + new Function, String>() { + @Override + public String call(MessageAndMetadata msgAndMd) { + return msgAndMd.message(); + } + } + ); + + // 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-beta/src/test/resources/log4j.properties b/external/kafka-beta/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..75e3b53a093f6 --- /dev/null +++ b/external/kafka-beta/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-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala new file mode 100644 index 0000000000000..8398178e9b79b --- /dev/null +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -0,0 +1,470 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.io.File +import java.util.Arrays +import java.util.concurrent.atomic.AtomicLong +import java.util.concurrent.ConcurrentLinkedQueue + +import scala.collection.JavaConverters._ +import scala.concurrent.duration._ +import scala.language.postfixOps + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.StringDecoder +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.concurrent.Eventually + +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.scheduler.rate.RateEstimator +import org.apache.spark.util.Utils + +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 smallest starting offset") { + val topics = Set("basic1", "basic2", "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( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + ssc = new StreamingContext(sparkConf, Milliseconds(200)) + val stream = withClue("Error creating direct stream") { + KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( + ssc, kafkaParams, topics) + } + + val allReceived = new ConcurrentLinkedQueue[(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) { + logInfo(s"${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.addAll(Arrays.asList(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.asScala.mkString("\n")) + } + ssc.stop() + } + + test("receiving from largest starting offset") { + val topic = "largest" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // 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, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.map { _._2 }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(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 = "offset" + val topicPartition = TopicAndPartition(topic, 0) + val data = Map("a" -> 10) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "largest" + ) + val kc = new KafkaCluster(kafkaParams) + def getLatestOffset(): Long = { + kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + } + + // 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, StringDecoder, StringDecoder, String]( + ssc, kafkaParams, Map(topicPartition -> 11L), + (m: MessageAndMetadata[String, String]) => m.message()) + } + assert( + stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] + .fromOffsets(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + stream.foreachRDD { rdd => collectedData.addAll(Arrays.asList(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 = "recovery" + kafkaTestUtils.createTopic(topic) + testDir = Utils.createTempDir() + + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + // 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, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } + val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => + Some(values.sum + state.getOrElse(0)) + } + ssc.checkpoint(testDir.getAbsolutePath) + + // This is to collect the raw data received from Kafka + kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => + val data = rdd.map { _._2 }.collect() + DirectKafkaStreamSuite.collectedData.addAll(Arrays.asList(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 = "report-test" + val data = Map("a" -> 7, "b" -> 9) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) + + val totalSent = data.values.sum + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + 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, StringDecoder, StringDecoder]( + ssc, kafkaParams, Set(topic)) + } + + val allReceived = new ConcurrentLinkedQueue[(String, String)] + + stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(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.asScala.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 = "backpressure" + val topicPartition = TopicAndPartition(topic, 0) + kafkaTestUtils.createTopic(topic) + val kafkaParams = Map( + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "auto.offset.reset" -> "smallest" + ) + + 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: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + val m = kc.getEarliestLeaderOffsets(Set(topicPartition)) + .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) + + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, kafkaParams, m, messageHandler) { + override protected[streaming] val rateController = + Some(new DirectKafkaRateController(id, estimator)) + } + } + + val collectedData = new ConcurrentLinkedQueue[Array[String]]() + + // Used for assertion failure messages. + def dataToString: String = + collectedData.asScala.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.add(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.asScala.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 ConcurrentLinkedQueue[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-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala new file mode 100644 index 0000000000000..d66830cbacdee --- /dev/null +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.util.Random + +import kafka.common.TopicAndPartition +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.SparkFunSuite + +class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { + private val topic = "kcsuitetopic" + Random.nextInt(10000) + private val topicAndPartition = TopicAndPartition(topic, 0) + private var kc: KafkaCluster = null + + private var kafkaTestUtils: KafkaTestUtils = _ + + override def beforeAll() { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) + kc = new KafkaCluster(Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress)) + } + + override def afterAll() { + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + test("metadata apis") { + val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) + val leaderAddress = s"${leader._1}:${leader._2}" + assert(leaderAddress === kafkaTestUtils.brokerAddress, "didn't get leader") + + val parts = kc.getPartitions(Set(topic)).right.get + assert(parts(topicAndPartition), "didn't get partitions") + + val err = kc.getPartitions(Set(topic + "BAD")) + assert(err.isLeft, "getPartitions for a nonexistant topic should be an error") + } + + test("leader offset apis") { + val earliest = kc.getEarliestLeaderOffsets(Set(topicAndPartition)).right.get + assert(earliest(topicAndPartition).offset === 0, "didn't get earliest") + + val latest = kc.getLatestLeaderOffsets(Set(topicAndPartition)).right.get + assert(latest(topicAndPartition).offset === 1, "didn't get latest") + } + + test("consumer offset apis") { + val group = "kcsuitegroup" + Random.nextInt(10000) + + val offset = Random.nextInt(10000) + + val set = kc.setConsumerOffsets(group, Map(topicAndPartition -> offset)) + assert(set.isRight, "didn't set consumer offsets") + + val get = kc.getConsumerOffsets(group, Set(topicAndPartition)).right.get + assert(get(topicAndPartition) === offset, "didn't get consumer offsets") + } +} diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala new file mode 100644 index 0000000000000..5e539c1d790cc --- /dev/null +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -0,0 +1,175 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import scala.util.Random + +import kafka.common.TopicAndPartition +import kafka.message.MessageAndMetadata +import kafka.serializer.StringDecoder +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark._ + +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"topicbasic-${Random.nextInt}" + kafkaTestUtils.createTopic(topic) + val messages = Array("the", "quick", "brown", "fox") + kafkaTestUtils.sendMessages(topic, messages) + + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}") + + val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) + + val rdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( + 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, StringDecoder, StringDecoder]( + 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, StringDecoder, StringDecoder]( + 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"topicboundary-${Random.nextInt}" + val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) + kafkaTestUtils.createTopic(topic) + + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, + "group.id" -> s"test-consumer-${Random.nextInt}") + + 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(kafkaParams("group.id"), rangesMap).fold( + err => throw new Exception(err.mkString("\n")), + _ => () + ) + + // this is the "0 messages" case + val rdd2 = getRdd(kc, Set(topic)) + // shouldn't get anything, since message is sent after rdd was defined + val sentOnlyOne = Map("d" -> 1) + + kafkaTestUtils.sendMessages(topic, sentOnlyOne) + + 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.getConsumerOffsets(groupId, topicPartitions).right.toOption.orElse( + kc.getEarliestLeaderOffsets(topicPartitions).right.toOption.map { offs => + offs.map(kv => kv._1 -> kv._2.offset) + } + ) + } + kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => + consumerOffsets(topicPartitions).flatMap { from => + kc.getLatestLeaderOffsets(topicPartitions).right.toOption.map { until => + val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) => + OffsetRange(tp.topic, tp.partition, fromOffset, until(tp).offset) + }.toArray + + val leaders = until.map { case (tp: TopicAndPartition, lo: KafkaCluster.LeaderOffset) => + tp -> Broker(lo.host, lo.port) + }.toMap + + KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder, String]( + sc, kc.kafkaParams, offsetRanges, leaders, + (mmd: MessageAndMetadata[String, String]) => s"${mmd.offset} ${mmd.message}") + } + } + } + } +} diff --git a/pom.xml b/pom.xml index 2148379896d35..2b8ddee81d5ab 100644 --- a/pom.xml +++ b/pom.xml @@ -114,6 +114,8 @@ launcher external/kafka external/kafka-assembly + external/kafka-beta + external/kafka-beta-assembly diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1ba6a075134c1..9960fccc79ba6 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -40,10 +40,10 @@ object BuildCommons { val streamingProjects@Seq( streaming, streamingFlumeSink, streamingFlume, streamingAkka, streamingKafka, streamingMqtt, - streamingTwitter, streamingZeromq + streamingTwitter, streamingZeromq, streamingKafkaBeta ) = Seq( "streaming", "streaming-flume-sink", "streaming-flume", "streaming-akka", "streaming-kafka", - "streaming-mqtt", "streaming-twitter", "streaming-zeromq" + "streaming-mqtt", "streaming-twitter", "streaming-zeromq", "streaming-kafka-beta" ).map(ProjectRef(buildLocation, _)) val allProjects@Seq( @@ -58,8 +58,8 @@ object BuildCommons { Seq("yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(assembly, examples, networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingMqttAssembly, streamingKinesisAslAssembly) = - Seq("assembly", "examples", "network-yarn", "streaming-flume-assembly", "streaming-kafka-assembly", "streaming-mqtt-assembly", "streaming-kinesis-asl-assembly") + val assemblyProjects@Seq(assembly, examples, networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafkaBetaAssembly, streamingMqttAssembly, streamingKinesisAslAssembly) = + Seq("assembly", "examples", "network-yarn", "streaming-flume-assembly", "streaming-kafka-assembly", "streaming-kafka-beta-assembly", "streaming-mqtt-assembly", "streaming-kinesis-asl-assembly") .map(ProjectRef(buildLocation, _)) val tools = ProjectRef(buildLocation, "tools") @@ -531,7 +531,7 @@ object Assembly { .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) => - if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-assembly") || mName.contains("streaming-mqtt-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { + if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-assembly") || mName.contains("streaming-kafka-beta-assembly") || mName.contains("streaming-mqtt-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { // This must match the same name used in maven (see external/kafka-assembly/pom.xml) s"${mName}-${v}.jar" } else { From d4e504a35f39831cd5c6ea077ea491307a5c2bda Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 4 Mar 2016 16:12:40 -0600 Subject: [PATCH 02/37] [SPARK-12177][Streaming][Kafka] WIP modify KafkaRDD to use cached new consumers --- .../streaming/kafka/CachedKafkaConsumer.scala | 141 ++++++++++++++++++ .../spark/streaming/kafka/KafkaRDD.scala | 137 ++++------------- 2 files changed, 173 insertions(+), 105 deletions(-) create mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala new file mode 100644 index 0000000000000..efc19d13cad25 --- /dev/null +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -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 + +import java.{ util => ju } + +import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord, KafkaConsumer } +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkConf + +/** Consumer of single topicpartition, intended for cached reuse. + * Underlying consumer is not threadsafe, so neither is this, + * but processing the same topicpartition and group id in multiple threads would be bad anyway. + */ +private[kafka] +class CachedKafkaConsumer[K, V] private( + val groupId: String, + val topic: String, + val partition: Int, + val kafkaConfig: ju.Map[String, Object]) { + + assert(groupId == kafkaConfig.get(ConsumerConfig.GROUP_ID_CONFIG), + "groupId used for cache key must match the groupId in kafkaConfig") + + val topicPartition = new TopicPartition(topic, partition) + + protected val consumer = { + val c = new KafkaConsumer[K, V](kafkaConfig) + val tps = new ju.ArrayList[TopicPartition]() + tps.add(topicPartition) + c.assign(tps) + c + } + + protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + + /** Get the record for the given offset, waiting up to timeout ms if IO is necessary. + * Sequential forward access will use buffers, but random access will be horribly inefficient. + */ + def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = { + if (!buffer.hasNext) { poll(timeout) } + + var record = buffer.next() + + if (record.offset != offset) { + seek(offset) + poll(timeout) + record = buffer.next() + assert(record.offset == offset, + s"Failed to get offset $offset after seeking to it") + } + + record + } + + private def seek(offset: Long): Unit = { + consumer.seek(topicPartition, offset) + } + + private def poll(timeout: Long): Unit = { + buffer = consumer.poll(timeout).records(topicPartition).iterator + } + +} + +private[kafka] +object CachedKafkaConsumer { + + private case class CacheKey(groupId: String, topic: String, partition: Int) + + // Don't want to depend on guava, don't want a cleanup thread, use a simple LinkedHashMap + private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]] = null + + /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ + def init(conf: SparkConf): Unit = CachedKafkaConsumer.synchronized { + if (null == cache) { + val initial = conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) + val max = conf.getInt("spark.streaming.kafka.consumer.cache.maxCapacity", 64) + val load = conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat + cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]](initial, load, true) { + override def removeEldestEntry( + entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer[_, _]]): Boolean = { + if (this.size > max) { + entry.getValue.consumer.close() + true + } else { + false + } + } + } + } + } + + /** Get a cached consumer for groupId, assigned to topic and partition. + * If matching consumer doesn't already exist, will be created using kafkaConfig. + */ + def get[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaConfig: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + CachedKafkaConsumer.synchronized { + val k = CacheKey(groupId, topic, partition) + val v = cache.get(k) + if (null == v) { + val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaConfig) + cache.put(k, c) + c + } else { + // any given topicpartition should have a consistent key and value type + v.asInstanceOf[CachedKafkaConsumer[K, V]] + } + } + + /** remove consumer for given groupId, topic, and partition, if it exists */ + def remove(groupId: String, topic: String, partition: Int): Unit = + CachedKafkaConsumer.synchronized { + val k = CacheKey(groupId, topic, partition) + val v = cache.get(k) + if (null != v) { + v.consumer.close() + cache.remove(k) + } + } +} diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 5e9ef2eadf90e..a7cac70a6625d 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -20,18 +20,15 @@ package org.apache.spark.streaming.kafka import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} -import kafka.api.{FetchRequestBuilder, FetchResponse} -import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.consumer.SimpleConsumer -import kafka.message.{MessageAndMetadata, MessageAndOffset} -import kafka.serializer.Decoder -import kafka.utils.VerifiableProperties +import java.{ util => ju } + +import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } +import org.apache.kafka.common.TopicPartition import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskContext} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.ExecutorCacheTaskLocation -import org.apache.spark.util.NextIterator /** * A batch-oriented interface for consuming from Kafka. @@ -47,18 +44,16 @@ import org.apache.spark.util.NextIterator class KafkaRDD[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, R: ClassTag] private[spark] ( sc: SparkContext, - kafkaParams: Map[String, String], + kafkaParams: ju.Map[String, Object], val offsetRanges: Array[OffsetRange], - leaders: Map[TopicAndPartition, (String, Int)], - messageHandler: MessageAndMetadata[K, V] => R + leaders: Map[TopicPartition, (String, Int)], + messageHandler: ConsumerRecord[K, V] => R ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { override def getPartitions: Array[Partition] = { offsetRanges.zipWithIndex.map { case (o, i) => - val (host, port) = leaders(TopicAndPartition(o.topic, o.partition)) + val (host, port) = leaders(new TopicPartition(o.topic, o.partition)) new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) }.toArray } @@ -137,16 +132,6 @@ class KafkaRDD[ 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)) @@ -161,91 +146,35 @@ class KafkaRDD[ private class KafkaRDDIterator( part: KafkaRDDPartition, - context: TaskContext) extends NextIterator[R] { - - context.addTaskCompletionListener{ context => closeIfNeeded() } + context: TaskContext) extends Iterator[R] { log.info(s"Computing topic ${part.topic}, partition ${part.partition} " + s"offsets ${part.fromOffset} -> ${part.untilOffset}") - val kc = new KafkaCluster(kafkaParams) - val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(kc.config.props) - .asInstanceOf[Decoder[K]] - val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(kc.config.props) - .asInstanceOf[Decoder[V]] - val consumer = connectLeader - var requestOffset = part.fromOffset - var iter: Iterator[MessageAndOffset] = null + val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] - // The idea is to use the provided preferred host, except on task retry attempts, - // to minimize number of kafka metadata requests - private def connectLeader: SimpleConsumer = { - if (context.attemptNumber > 0) { - kc.connectLeader(part.topic, part.partition).fold( - errs => throw new SparkException( - s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " + - errs.mkString("\n")), - consumer => consumer - ) - } else { - kc.connect(part.host, part.port) - } - } + val pollTimeout = sparkContext.getConf.getLong("spark.streaming.kafka.consumer.poll.ms", 10) - private def handleFetchErr(resp: FetchResponse) { - if (resp.hasError) { - val err = resp.errorCode(part.topic, part.partition) - if (err == ErrorMapping.LeaderNotAvailableCode || - err == ErrorMapping.NotLeaderForPartitionCode) { - log.error(s"Lost leader for topic ${part.topic} partition ${part.partition}, " + - s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms") - Thread.sleep(kc.config.refreshLeaderBackoffMs) - } - // Let normal rdd retry sort out reconnect attempts - throw ErrorMapping.exceptionFor(err) + val consumer = { + CachedKafkaConsumer.init(sparkContext.getConf) + if (context.attemptNumber > 0) { + // just in case the prior attempt failure was cache related + CachedKafkaConsumer.remove(groupId, part.topic, part.partition) } + CachedKafkaConsumer.get[K, V](groupId, part.topic, part.partition, kafkaParams) } - private def fetchBatch: Iterator[MessageAndOffset] = { - val req = new FetchRequestBuilder() - .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes) - .build() - val resp = consumer.fetch(req) - handleFetchErr(resp) - // kafka may return a batch that starts before the requested offset - resp.messageSet(part.topic, part.partition) - .iterator - .dropWhile(_.offset < requestOffset) - } + var requestOffset = part.fromOffset - override def close(): Unit = { - if (consumer != null) { - consumer.close() - } - } + override def hasNext(): Boolean = requestOffset < part.untilOffset - override def getNext(): R = { - if (iter == null || !iter.hasNext) { - iter = fetchBatch - } - if (!iter.hasNext) { - assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) - finished = true - null.asInstanceOf[R] - } else { - val item = iter.next() - if (item.offset >= part.untilOffset) { - assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) - finished = true - null.asInstanceOf[R] - } else { - requestOffset = item.nextOffset - messageHandler(new MessageAndMetadata( - part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder)) - } - } + override def next(): R = { + assert(hasNext(), "Can't call getNext() once untilOffset has been reached") + // XXX TODO is messageHandler even useful any more? + // Don't think it can catch serialization problems with the new consumer in an efficient way + val r = messageHandler(consumer.get(requestOffset, pollTimeout)) + requestOffset += 1 + r } } } @@ -268,15 +197,13 @@ object KafkaRDD { def apply[ K: ClassTag, V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, R: ClassTag]( sc: SparkContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - untilOffsets: Map[TopicAndPartition, LeaderOffset], - messageHandler: MessageAndMetadata[K, V] => R - ): KafkaRDD[K, V, U, T, R] = { + kafkaParams: ju.Map[String, Object], + fromOffsets: Map[TopicPartition, Long], + untilOffsets: Map[TopicPartition, LeaderOffset], + messageHandler: ConsumerRecord[K, V] => R + ): KafkaRDD[K, V, R] = { val leaders = untilOffsets.map { case (tp, lo) => tp -> (lo.host, lo.port) }.toMap @@ -286,6 +213,6 @@ object KafkaRDD { OffsetRange(tp.topic, tp.partition, fo, uo.offset) }.toArray - new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) + new KafkaRDD[K, V, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) } } From 37cca1ea2acfbad81d0466eb6bd49352570194da Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Mon, 7 Mar 2016 16:21:41 -0600 Subject: [PATCH 03/37] [SPARK-12177][Streaming][Kafka] WIP modify DirectKafkaInputDStream to use new consumer --- .../streaming/kafka/CachedKafkaConsumer.scala | 14 +- .../kafka/DirectKafkaInputDStream.scala | 264 ++++++++--- .../spark/streaming/kafka/KafkaCluster.scala | 425 ------------------ .../spark/streaming/kafka/KafkaRDD.scala | 100 ++--- .../streaming/kafka/KafkaRDDPartition.scala | 13 +- .../spark/streaming/kafka/KafkaUtils.scala | 14 +- .../spark/streaming/kafka/OffsetRange.scala | 16 +- .../streaming/kafka/KafkaClusterSuite.scala | 81 ---- 8 files changed, 269 insertions(+), 658 deletions(-) delete mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala delete mode 100644 external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index efc19d13cad25..4ce52a6ca9c3b 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -33,15 +33,15 @@ class CachedKafkaConsumer[K, V] private( val groupId: String, val topic: String, val partition: Int, - val kafkaConfig: ju.Map[String, Object]) { + val kafkaParams: ju.Map[String, Object]) { - assert(groupId == kafkaConfig.get(ConsumerConfig.GROUP_ID_CONFIG), - "groupId used for cache key must match the groupId in kafkaConfig") + assert(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG), + "groupId used for cache key must match the groupId in kafkaParams") val topicPartition = new TopicPartition(topic, partition) protected val consumer = { - val c = new KafkaConsumer[K, V](kafkaConfig) + val c = new KafkaConsumer[K, V](kafkaParams) val tps = new ju.ArrayList[TopicPartition]() tps.add(topicPartition) c.assign(tps) @@ -108,18 +108,18 @@ object CachedKafkaConsumer { } /** Get a cached consumer for groupId, assigned to topic and partition. - * If matching consumer doesn't already exist, will be created using kafkaConfig. + * If matching consumer doesn't already exist, will be created using kafkaParams. */ def get[K, V]( groupId: String, topic: String, partition: Int, - kafkaConfig: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = CachedKafkaConsumer.synchronized { val k = CacheKey(groupId, topic, partition) val v = cache.get(k) if (null == v) { - val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaConfig) + val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) cache.put(k, c) c } else { diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 54d8c8b03f206..407b0fe86fdae 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -21,17 +21,21 @@ import scala.annotation.tailrec import scala.collection.mutable import scala.reflect.ClassTag -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.Decoder +import java.{ util => ju } + +import org.apache.kafka.clients.consumer.{ + ConsumerConfig, ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer +} +import org.apache.kafka.common.{ PartitionInfo, TopicPartition } import org.apache.spark.{Logging, SparkException} import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} import org.apache.spark.streaming.scheduler.rate.RateEstimator +import scala.collection.JavaConverters._ + /** * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where * each given Kafka topic/partition corresponds to an RDD partition. @@ -41,30 +45,142 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator * 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 + * @param kafkaParams Kafka * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), + * 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 into the desired type */ -private[streaming] -class DirectKafkaInputDStream[ - K: ClassTag, - V: ClassTag, - U <: Decoder[K]: ClassTag, - T <: Decoder[V]: ClassTag, - R: ClassTag]( + +class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( _ssc: StreamingContext, - val kafkaParams: Map[String, String], - val fromOffsets: Map[TopicAndPartition, Long], - messageHandler: MessageAndMetadata[K, V] => R - ) extends InputDStream[R](_ssc) with Logging { - val maxRetries = context.sparkContext.getConf.getInt( - "spark.streaming.kafka.maxRetries", 1) + val driverKafkaParams: ju.Map[String, Object], + val executorKafkaParams: ju.Map[String, Object], + preferredHosts: ju.Map[TopicPartition, String] + ) extends InputDStream[ConsumerRecord[K,V]](_ssc) with Logging { + + import DirectKafkaInputDStream.{ + PartitionAssignment, Assigned, Subscribed, PatternSubscribed, Unassigned + } + + assert(1 == + driverKafkaParams.get(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG).asInstanceOf[Int], + ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG + + " must be set to 1 for driver kafka params, because the driver should not consume messages") + + assert(false == + driverKafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + + " must be set to false for driver kafka params, else offsets may commit before processing") + + @transient private var kc: KafkaConsumer[K, V] = null + private var partitionAssignment: PartitionAssignment = Unassigned + protected def consumer(): KafkaConsumer[K, V] = this.synchronized { + if (null == kc) { + kc = new KafkaConsumer(driverKafkaParams) + assignPartitions(partitionAssignment) + } + kc + } + consumer() + + private def listenerFor(className: String): ConsumerRebalanceListener = + Class.forName(className) + .newInstance() + .asInstanceOf[ConsumerRebalanceListener] + + private def assignPartitions(pa: PartitionAssignment): Unit = this.synchronized { + // using kc directly because consumer() calls this method + pa match { + case Assigned(partitions) => + kc.assign(partitions) + case Subscribed(topics, className) => + kc.subscribe(topics, listenerFor(className)) + case PatternSubscribed(pattern, className) => + kc.subscribe(pattern, listenerFor(className)) + case Unassigned => + } + + this.partitionAssignment = pa + } + + /** Manually assign a list of partitions */ + def assign(partitions: ju.List[TopicPartition]): Unit = { + assignPartitions(Assigned(partitions)) + } + + /** Subscribe to the given list of topics to get dynamically assigned partitions */ + def subscribe(topics: ju.List[String]): Unit = { + assignPartitions(Subscribed(topics)) + } + + /** Subscribe to the given list of topics to get dynamically assigned partitions */ + def subscribe( + topics: ju.List[String], + consumerRebalanceListenerClassName: String): Unit = { + assignPartitions(Subscribed(topics, consumerRebalanceListenerClassName)) + } + + /** Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + */ + def subscribe(pattern: ju.regex.Pattern): Unit = { + assignPartitions(PatternSubscribed(pattern)) + } + + /** Subscribe to all topics matching specified pattern to get dynamically assigned partitions. + * The pattern matching will be done periodically against topics existing at the time of check. + */ + def subscribe( + pattern: ju.regex.Pattern, + consumerRebalanceListenerClassName: String): Unit = { + assignPartitions(PatternSubscribed(pattern, consumerRebalanceListenerClassName)) + } + + /** Get the set of partitions currently assigned to the underlying consumer */ + def assignment(): ju.Set[TopicPartition] = this.synchronized { + consumer.assignment() + } + + /** Get metadata about the partitions for a given topic. */ + def partitionsFor(topic: String): ju.List[PartitionInfo] = this.synchronized { + consumer.partitionsFor(topic) + } + + private val pollTimeout = + context.sparkContext.getConf.getLong("spark.streaming.kafka.consumer.poll.ms", 10) + /** Necessary to fetch metadata and update subscriptions, won't actually return useful data */ + def poll(): Unit = poll(pollTimeout) + + /** Necessary to fetch metadata and update subscriptions, won't actually return useful data */ + def poll(timeout: Long): Unit = this.synchronized { + consumer.poll(pollTimeout) + } + + // TODO is there a better way to distinguish between + // - want to use leader brokers (null map) + // - don't care, use consistent executor (empty map) + // - want to use specific hosts (non-null, non-empty map) + private def getPreferredHosts: ju.Map[TopicPartition, String] = { + if (null != preferredHosts) { + preferredHosts + } else { + val result = new ju.HashMap[TopicPartition, String]() + val hosts = new ju.HashMap[TopicPartition, String]() + val assignments = assignment().iterator() + while (assignments.hasNext()) { + val a = assignments.next() + if (null == hosts.get(a)) { + val infos = partitionsFor(a.topic).iterator() + while (infos.hasNext()) { + val i = infos.next() + hosts.put(new TopicPartition(i.topic(), i.partition()), i.leader.host()) + } + } + result.put(a, hosts.get(a)) + } + result + } + } // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") private[streaming] override def name: String = s"Kafka direct stream [$id]" @@ -85,8 +201,6 @@ class DirectKafkaInputDStream[ } } - protected val kc = new KafkaCluster(kafkaParams) - private val maxRateLimitPerPartition: Int = context.sparkContext.getConf.getInt( "spark.streaming.kafka.maxRatePerPartition", 0) protected def maxMessagesPerPartition: Option[Long] = { @@ -111,46 +225,37 @@ class DirectKafkaInputDStream[ } } - protected var currentOffsets = fromOffsets - - @tailrec - protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { - val o = kc.getLatestLeaderOffsets(currentOffsets.keySet) - // Either.fold would confuse @tailrec, do it manually - if (o.isLeft) { - val err = o.left.get.toString - if (retries <= 0) { - throw new SparkException(err) - } else { - log.error(err) - Thread.sleep(kc.config.refreshLeaderBackoffMs) - latestLeaderOffsets(retries - 1) - } - } else { - o.right.get - } + protected var currentOffsets = Map[TopicPartition, Long]() + + protected def latestOffsets(): Map[TopicPartition, Long] = this.synchronized { + val c = consumer + c.seekToEnd() + c.assignment().asScala.map { tp => + tp -> c.position(tp) + }.toMap } // limits the maximum number of messages per partition protected def clamp( - leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = { + offsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { maxMessagesPerPartition.map { mmp => - leaderOffsets.map { case (tp, lo) => - tp -> lo.copy(offset = Math.min(currentOffsets(tp) + mmp, lo.offset)) + offsets.map { case (tp, o) => + tp -> Math.min(currentOffsets(tp) + mmp, o) } - }.getOrElse(leaderOffsets) + }.getOrElse(offsets) } - override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = { - val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) - val rdd = KafkaRDD[K, V, U, T, R]( - context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) + override def compute(validTime: Time): Option[KafkaRDD[K, V]] = { + val untilOffsets = clamp(latestOffsets()) + val offsetRanges = untilOffsets.map { case (tp, uo) => + val fo = currentOffsets.getOrElse(tp, uo) + OffsetRange(tp.topic, tp.partition, fo, uo) + } + + val rdd = KafkaRDD[K, V]( + context.sparkContext, executorKafkaParams, offsetRanges.toArray, getPreferredHosts) // 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.offset) - } val description = offsetRanges.filter { offsetRange => // Don't display empty ranges. offsetRange.fromOffset != offsetRange.untilOffset @@ -165,14 +270,23 @@ class DirectKafkaInputDStream[ val inputInfo = StreamInputInfo(id, rdd.count, metadata) ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) - currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) + currentOffsets = untilOffsets Some(rdd) } - override def start(): Unit = { + override def start(): Unit = this.synchronized { + assert(partitionAssignment != Unassigned, "Must call subscribe or assign before starting") + val c = consumer + c.poll(pollTimeout) + currentOffsets = c.assignment().asScala.map { tp => + tp -> c.position(tp) + }.toMap } - def stop(): Unit = { + override def stop(): Unit = { + this.synchronized { + consumer.close() + } } private[streaming] @@ -184,7 +298,7 @@ class DirectKafkaInputDStream[ override def update(time: Time) { batchForTime.clear() generatedRDDs.foreach { kv => - val a = kv._2.asInstanceOf[KafkaRDD[K, V, U, T, R]].offsetRanges.map(_.toTuple).toArray + val a = kv._2.asInstanceOf[KafkaRDD[K, V]].offsetRanges.map(_.toTuple).toArray batchForTime += kv._1 -> a } } @@ -192,14 +306,12 @@ class DirectKafkaInputDStream[ 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 - val leaders = KafkaCluster.checkErrors(kc.findLeaders(topics)) + poll() 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, U, T, R]( - context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, messageHandler) + generatedRDDs += t -> KafkaRDD[K, V]( + context.sparkContext, executorKafkaParams, b.map(OffsetRange(_)), getPreferredHosts) } } } @@ -212,3 +324,25 @@ class DirectKafkaInputDStream[ override def publish(rate: Long): Unit = () } } + +object DirectKafkaInputDStream { + protected val defaultListener = + "org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener" + /** There are several different ways of specifying partition assignment, + * and they need to be able to survive checkpointing + */ + protected sealed trait PartitionAssignment extends Serializable + /** manual assignment via consumer.assign() */ + protected case class Assigned(partitions: ju.List[TopicPartition]) extends PartitionAssignment + /** dynamic subscription to list of topics via consumer.subscribe */ + protected case class Subscribed( + topics: ju.List[String], + consumerRebalanceListenerClassName: String = defaultListener) extends PartitionAssignment + /** dynamic subscription to topics matching pattern via consumer.subscribe */ + protected case class PatternSubscribed( + pattern: ju.regex.Pattern, + consumerRebalanceListenerClassName: String = defaultListener) extends PartitionAssignment + /** Not yet assigned */ + protected case object Unassigned extends PartitionAssignment + +} diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala deleted file mode 100644 index 8a66621a3125c..0000000000000 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ /dev/null @@ -1,425 +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 - -import java.util.Properties - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.util.Random -import scala.util.control.NonFatal - -import kafka.api._ -import kafka.common.{ErrorMapping, OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} -import kafka.consumer.{ConsumerConfig, SimpleConsumer} - -import org.apache.spark.SparkException -import org.apache.spark.annotation.DeveloperApi - -/** - * :: DeveloperApi :: - * Convenience methods for interacting with a Kafka cluster. - * See - * A Guide To The Kafka Protocol for more details on individual api calls. - * @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 - */ -@DeveloperApi -class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { - import KafkaCluster.{Err, LeaderOffset, SimpleConsumerConfig} - - // ConsumerConfig isn't serializable - @transient private var _config: SimpleConsumerConfig = null - - def config: SimpleConsumerConfig = this.synchronized { - if (_config == null) { - _config = SimpleConsumerConfig(kafkaParams) - } - _config - } - - def connect(host: String, port: Int): SimpleConsumer = - new SimpleConsumer(host, port, config.socketTimeoutMs, - config.socketReceiveBufferBytes, config.clientId) - - def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] = - findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2)) - - // Metadata api - // scalastyle:off - // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI - // scalastyle:on - - def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = { - val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, - 0, config.clientId, Seq(topic)) - val errs = new Err - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp: TopicMetadataResponse = consumer.send(req) - resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata => - tm.partitionsMetadata.find(_.partitionId == partition) - }.foreach { pm: PartitionMetadata => - pm.leader.foreach { leader => - return Right((leader.host, leader.port)) - } - } - } - Left(errs) - } - - def findLeaders( - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, (String, Int)]] = { - val topics = topicAndPartitions.map(_.topic) - val response = getPartitionMetadata(topics).right - val answer = response.flatMap { tms: Set[TopicMetadata] => - val leaderMap = tms.flatMap { tm: TopicMetadata => - tm.partitionsMetadata.flatMap { pm: PartitionMetadata => - val tp = TopicAndPartition(tm.topic, pm.partitionId) - if (topicAndPartitions(tp)) { - pm.leader.map { l => - tp -> (l.host -> l.port) - } - } else { - None - } - } - }.toMap - - if (leaderMap.keys.size == topicAndPartitions.size) { - Right(leaderMap) - } else { - val missing = topicAndPartitions.diff(leaderMap.keySet) - val err = new Err - err.append(new SparkException(s"Couldn't find leaders for ${missing}")) - Left(err) - } - } - answer - } - - def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = { - getPartitionMetadata(topics).right.map { r => - r.flatMap { tm: TopicMetadata => - tm.partitionsMetadata.map { pm: PartitionMetadata => - TopicAndPartition(tm.topic, pm.partitionId) - } - } - } - } - - def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = { - val req = TopicMetadataRequest( - TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq) - val errs = new Err - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp: TopicMetadataResponse = consumer.send(req) - val respErrs = resp.topicsMetadata.filter(m => m.errorCode != ErrorMapping.NoError) - - if (respErrs.isEmpty) { - return Right(resp.topicsMetadata.toSet) - } else { - respErrs.foreach { m => - val cause = ErrorMapping.exceptionFor(m.errorCode) - val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" - errs.append(new SparkException(msg, cause)) - } - } - } - Left(errs) - } - - // Leader offset api - // scalastyle:off - // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI - // scalastyle:on - - def getLatestLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = - getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime) - - def getEarliestLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = - getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime) - - def getLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition], - before: Long - ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = { - getLeaderOffsets(topicAndPartitions, before, 1).right.map { r => - r.map { kv => - // mapValues isnt serializable, see SI-7005 - kv._1 -> kv._2.head - } - } - } - - private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] = - m.groupBy(_._2).map { kv => - kv._1 -> kv._2.keys.toSeq - } - - def getLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition], - before: Long, - maxNumOffsets: Int - ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = { - findLeaders(topicAndPartitions).right.flatMap { tpToLeader => - val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader) - val leaders = leaderToTp.keys - var result = Map[TopicAndPartition, Seq[LeaderOffset]]() - val errs = new Err - withBrokers(leaders, errs) { consumer => - val partitionsToGetOffsets: Seq[TopicAndPartition] = - leaderToTp((consumer.host, consumer.port)) - val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition => - tp -> PartitionOffsetRequestInfo(before, maxNumOffsets) - }.toMap - val req = OffsetRequest(reqMap) - val resp = consumer.getOffsetsBefore(req) - val respMap = resp.partitionErrorAndOffsets - partitionsToGetOffsets.foreach { tp: TopicAndPartition => - respMap.get(tp).foreach { por: PartitionOffsetsResponse => - if (por.error == ErrorMapping.NoError) { - if (por.offsets.nonEmpty) { - result += tp -> por.offsets.map { off => - LeaderOffset(consumer.host, consumer.port, off) - } - } else { - errs.append(new SparkException( - s"Empty offsets for ${tp}, is ${before} before log beginning?")) - } - } else { - errs.append(ErrorMapping.exceptionFor(por.error)) - } - } - } - if (result.keys.size == topicAndPartitions.size) { - return Right(result) - } - } - val missing = topicAndPartitions.diff(result.keySet) - errs.append(new SparkException(s"Couldn't find leader offsets for ${missing}")) - Left(errs) - } - } - - // Consumer offset api - // scalastyle:off - // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI - // scalastyle:on - - // this 0 here indicates api version, in this case the original ZK backed api. - private def defaultConsumerApiVersion: Short = 0 - - /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ - def getConsumerOffsets( - groupId: String, - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, Long]] = - getConsumerOffsets(groupId, topicAndPartitions, defaultConsumerApiVersion) - - def getConsumerOffsets( - groupId: String, - topicAndPartitions: Set[TopicAndPartition], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, Long]] = { - getConsumerOffsetMetadata(groupId, topicAndPartitions, consumerApiVersion).right.map { r => - r.map { kv => - kv._1 -> kv._2.offset - } - } - } - - /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ - def getConsumerOffsetMetadata( - groupId: String, - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = - getConsumerOffsetMetadata(groupId, topicAndPartitions, defaultConsumerApiVersion) - - def getConsumerOffsetMetadata( - groupId: String, - topicAndPartitions: Set[TopicAndPartition], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = { - var result = Map[TopicAndPartition, OffsetMetadataAndError]() - val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq, consumerApiVersion) - val errs = new Err - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp = consumer.fetchOffsets(req) - val respMap = resp.requestInfo - val needed = topicAndPartitions.diff(result.keySet) - needed.foreach { tp: TopicAndPartition => - respMap.get(tp).foreach { ome: OffsetMetadataAndError => - if (ome.error == ErrorMapping.NoError) { - result += tp -> ome - } else { - errs.append(ErrorMapping.exceptionFor(ome.error)) - } - } - } - if (result.keys.size == topicAndPartitions.size) { - return Right(result) - } - } - val missing = topicAndPartitions.diff(result.keySet) - errs.append(new SparkException(s"Couldn't find consumer offsets for ${missing}")) - Left(errs) - } - - /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ - def setConsumerOffsets( - groupId: String, - offsets: Map[TopicAndPartition, Long] - ): Either[Err, Map[TopicAndPartition, Short]] = - setConsumerOffsets(groupId, offsets, defaultConsumerApiVersion) - - def setConsumerOffsets( - groupId: String, - offsets: Map[TopicAndPartition, Long], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, Short]] = { - val meta = offsets.map { kv => - kv._1 -> OffsetAndMetadata(kv._2) - } - setConsumerOffsetMetadata(groupId, meta, consumerApiVersion) - } - - /** Requires Kafka >= 0.8.1.1. Defaults to the original ZooKeeper backed api version. */ - def setConsumerOffsetMetadata( - groupId: String, - metadata: Map[TopicAndPartition, OffsetAndMetadata] - ): Either[Err, Map[TopicAndPartition, Short]] = - setConsumerOffsetMetadata(groupId, metadata, defaultConsumerApiVersion) - - def setConsumerOffsetMetadata( - groupId: String, - metadata: Map[TopicAndPartition, OffsetAndMetadata], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, Short]] = { - var result = Map[TopicAndPartition, Short]() - val req = OffsetCommitRequest(groupId, metadata, consumerApiVersion) - val errs = new Err - val topicAndPartitions = metadata.keySet - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp = consumer.commitOffsets(req) - val respMap = resp.commitStatus - val needed = topicAndPartitions.diff(result.keySet) - needed.foreach { tp: TopicAndPartition => - respMap.get(tp).foreach { err: Short => - if (err == ErrorMapping.NoError) { - result += tp -> err - } else { - errs.append(ErrorMapping.exceptionFor(err)) - } - } - } - if (result.keys.size == topicAndPartitions.size) { - return Right(result) - } - } - val missing = topicAndPartitions.diff(result.keySet) - errs.append(new SparkException(s"Couldn't set offsets for ${missing}")) - Left(errs) - } - - // Try a call against potentially multiple brokers, accumulating errors - private def withBrokers(brokers: Iterable[(String, Int)], errs: Err) - (fn: SimpleConsumer => Any): Unit = { - brokers.foreach { hp => - var consumer: SimpleConsumer = null - try { - consumer = connect(hp._1, hp._2) - fn(consumer) - } catch { - case NonFatal(e) => - errs.append(e) - } finally { - if (consumer != null) { - consumer.close() - } - } - } - } -} - -@DeveloperApi -object KafkaCluster { - 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 - ) - } - - case class LeaderOffset(host: String, port: Int, offset: Long) - - /** - * High-level kafka consumers connect to ZK. ConsumerConfig assumes this use case. - * Simple consumers connect directly to brokers, but need many of the same configs. - * This subclass won't warn about missing ZK params, or presence of broker params. - */ - class SimpleConsumerConfig private(brokers: String, originalProps: Properties) - extends ConsumerConfig(originalProps) { - val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => - val hpa = hp.split(":") - if (hpa.size == 1) { - throw new SparkException(s"Broker not in the correct format of : [$brokers]") - } - (hpa(0), hpa(1).toInt) - } - } - - object SimpleConsumerConfig { - /** - * Make a consumer config without requiring group.id or zookeeper.connect, - * since communicating with brokers also needs common settings such as timeout - */ - def apply(kafkaParams: Map[String, String]): SimpleConsumerConfig = { - // These keys are from other pre-existing kafka configs for specifying brokers, accept either - val brokers = kafkaParams.get("metadata.broker.list") - .orElse(kafkaParams.get("bootstrap.servers")) - .getOrElse(throw new SparkException( - "Must specify metadata.broker.list or bootstrap.servers")) - - val props = new Properties() - kafkaParams.foreach { case (key, value) => - // prevent warnings on parameters ConsumerConfig doesn't know about - if (key != "metadata.broker.list" && key != "bootstrap.servers") { - props.put(key, value) - } - } - - Seq("zookeeper.connect", "group.id").foreach { s => - if (!props.containsKey(s)) { - props.setProperty(s, "") - } - } - - new SimpleConsumerConfig(brokers, props) - } - } -} diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index a7cac70a6625d..1613f9cc5a2be 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -34,27 +34,34 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation * 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 + * @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 - * @param messageHandler function for translating each message into the desired type */ class KafkaRDD[ K: ClassTag, - V: ClassTag, - R: ClassTag] private[spark] ( + V: ClassTag] private ( sc: SparkContext, - kafkaParams: ju.Map[String, Object], + val kafkaParams: ju.Map[String, Object], val offsetRanges: Array[OffsetRange], - leaders: Map[TopicPartition, (String, Int)], - messageHandler: ConsumerRecord[K, V] => R - ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { + val preferredHosts: ju.Map[TopicPartition, String] +) extends RDD[ConsumerRecord[K, V]](sc, Nil) with Logging with HasOffsetRanges { + + assert("none" == + kafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).asInstanceOf[String], + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + + " must be set to none for executor kafka params, else messages may not match offsetRange") + + assert(false == + kafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + + " must be set to false for executor kafka params, else offsets may commit before processing") + override def getPartitions: Array[Partition] = { offsetRanges.zipWithIndex.map { case (o, i) => - val (host, port) = leaders(new TopicPartition(o.topic, o.partition)) - new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) + new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) }.toArray } @@ -70,13 +77,13 @@ class KafkaRDD[ override def isEmpty(): Boolean = count == 0L - override def take(num: Int): Array[R] = { + override def take(num: Int): Array[ConsumerRecord[K, V]] = { val nonEmptyPartitions = this.partitions .map(_.asInstanceOf[KafkaRDDPartition]) .filter(_.count > 0) if (num < 1 || nonEmptyPartitions.size < 1) { - return new Array[R](0) + return new Array[ConsumerRecord[K, V]](0) } // Determine in advance how many messages need to be taken from each partition @@ -90,11 +97,12 @@ class KafkaRDD[ } } - val buf = new ArrayBuffer[R] + val buf = new ArrayBuffer[ConsumerRecord[K, V]] val res = context.runJob( this, - (tc: TaskContext, it: Iterator[R]) => it.take(parts(tc.partitionId)).toArray, - parts.keys.toArray) + (tc: TaskContext, it: Iterator[ConsumerRecord[K, V]]) => + it.take(parts(tc.partitionId)).toArray, parts.keys.toArray + ) res.foreach(buf ++= _) buf.toArray } @@ -115,13 +123,14 @@ class KafkaRDD[ private def floorMod(a: Int, b: Int): Int = ((a % b) + b) % b override def getPreferredLocations(thePart: Partition): Seq[String] = { - // want lead broker if an executor is running on it, otherwise stable exec to use caching - // TODO what if broker host is ip and executor is name, or vice versa + // TODO what about hosts specified by ip vs name val part = thePart.asInstanceOf[KafkaRDDPartition] val allExecs = executors() - val brokerExecs = allExecs.filter(_.host == part.host) - val execs = if (brokerExecs.isEmpty) allExecs else brokerExecs - val index = floorMod(part.topic.hashCode * 41 + part.partition.hashCode, execs.length) + val tp = part.topicPartition + val prefHost = preferredHosts.get(tp) + val prefExecs = if (null == prefHost) allExecs else allExecs.filter(_.host == prefHost) + val execs = if (prefExecs.isEmpty) allExecs else prefExecs + val index = floorMod(tp.hashCode, execs.length) val chosen = execs(index) Seq(chosen.toString) @@ -132,7 +141,7 @@ class KafkaRDD[ s"for topic ${part.topic} partition ${part.partition}. " + "You either provided an invalid fromOffset, or the Kafka topic has been damaged" - override def compute(thePart: Partition, context: TaskContext): Iterator[R] = { + override def compute(thePart: Partition, context: TaskContext): Iterator[ConsumerRecord[K, V]] = { val part = thePart.asInstanceOf[KafkaRDDPartition] assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) if (part.fromOffset == part.untilOffset) { @@ -146,7 +155,7 @@ class KafkaRDD[ private class KafkaRDDIterator( part: KafkaRDDPartition, - context: TaskContext) extends Iterator[R] { + context: TaskContext) extends Iterator[ConsumerRecord[K, V]] { log.info(s"Computing topic ${part.topic}, partition ${part.partition} " + s"offsets ${part.fromOffset} -> ${part.untilOffset}") @@ -168,51 +177,26 @@ class KafkaRDD[ override def hasNext(): Boolean = requestOffset < part.untilOffset - override def next(): R = { + override def next(): ConsumerRecord[K, V] = { assert(hasNext(), "Can't call getNext() once untilOffset has been reached") - // XXX TODO is messageHandler even useful any more? - // Don't think it can catch serialization problems with the new consumer in an efficient way - val r = messageHandler(consumer.get(requestOffset, pollTimeout)) + val r = consumer.get(requestOffset, pollTimeout) requestOffset += 1 r } } } -private[kafka] object KafkaRDD { - import KafkaCluster.LeaderOffset - - /** - * @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 - * @param messageHandler function for translating each message into the desired type - */ + def apply[ K: ClassTag, - V: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: ju.Map[String, Object], - fromOffsets: Map[TopicPartition, Long], - untilOffsets: Map[TopicPartition, LeaderOffset], - messageHandler: ConsumerRecord[K, V] => R - ): KafkaRDD[K, V, R] = { - val leaders = untilOffsets.map { case (tp, lo) => - tp -> (lo.host, lo.port) - }.toMap - - val offsetRanges = fromOffsets.map { case (tp, fo) => - val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo.offset) - }.toArray + V: ClassTag]( + sc: SparkContext, + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + preferredHosts: ju.Map[TopicPartition, String] + ): KafkaRDD[K, V] = { - new KafkaRDD[K, V, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) + new KafkaRDD[K, V](sc, kafkaParams, offsetRanges, preferredHosts) } } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala index a660d2a00c35d..2b8a3d6830408 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala @@ -17,14 +17,15 @@ package org.apache.spark.streaming.kafka +import org.apache.kafka.common.TopicPartition + 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 host preferred kafka host, i.e. the leader at the time the rdd was created - * @param port preferred kafka host's port */ private[kafka] class KafkaRDDPartition( @@ -32,10 +33,12 @@ class KafkaRDDPartition( val topic: String, val partition: Int, val fromOffset: Long, - val untilOffset: Long, - val host: String, - val port: Int + val untilOffset: Long ) extends Partition { /** Number of messages this partition refers to */ def count(): Long = untilOffset - fromOffset + + /** Kafka TopicPartition object, for convenience */ + def topicPartition(): TopicPartition = new TopicPartition(topic, partition) + } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index c94b1855e2821..af8f1863ecea9 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -110,9 +110,7 @@ object KafkaUtils { */ def createRDD[ K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag]( + V: ClassTag]( sc: SparkContext, kafkaParams: Map[String, String], offsetRanges: Array[OffsetRange] @@ -121,7 +119,7 @@ object KafkaUtils { val kc = new KafkaCluster(kafkaParams) val leaders = leadersForRanges(kc, offsetRanges) checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) + new KafkaRDD[K, V, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) } /** @@ -149,8 +147,6 @@ object KafkaUtils { def createRDD[ K: ClassTag, V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag, R: ClassTag]( sc: SparkContext, kafkaParams: Map[String, String], @@ -205,7 +201,7 @@ object KafkaUtils { implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - new JavaPairRDD(createRDD[K, V, KD, VD]( + new JavaPairRDD(createRDD[K, V]( jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges)) } @@ -592,14 +588,14 @@ private[kafka] class KafkaUtilsPythonHelper { def offsetRangesOfKafkaRDD(rdd: RDD[_]): JList[OffsetRange] = { val parentRDDs = rdd.getNarrowAncestors - val kafkaRDDs = parentRDDs.filter(rdd => rdd.isInstanceOf[KafkaRDD[_, _, _, _, _]]) + 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[_, _, _, _, _]] + val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _]] kafkaRDD.offsetRanges.toSeq.asJava } } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index d9b856e4697a0..72d14508461ce 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.kafka -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 @@ -35,7 +35,7 @@ trait HasOffsetRanges { } /** - * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class + * Represents a range of offsets from a single Kafka TopicPartition. Instances of this class * can be created with `OffsetRange.create()`. * @param topic Kafka topic name * @param partition Kafka partition id @@ -49,8 +49,8 @@ final class OffsetRange private( val untilOffset: Long) extends Serializable { import OffsetRange.OffsetRangeTuple - /** Kafka TopicAndPartition object, for convenience */ - def topicAndPartition(): TopicAndPartition = TopicAndPartition(topic, partition) + /** Kafka TopicPartition object, for convenience */ + def topicPartition(): TopicPartition = new TopicPartition(topic, partition) /** Number of messages this OffsetRange refers to */ def count(): Long = untilOffset - fromOffset @@ -85,19 +85,19 @@ object OffsetRange { new OffsetRange(topic, partition, fromOffset, untilOffset) def create( - topicAndPartition: TopicAndPartition, + topicPartition: TopicPartition, fromOffset: Long, untilOffset: Long): OffsetRange = - new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) + new OffsetRange(topicPartition.topic, topicPartition.partition, fromOffset, untilOffset) def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = new OffsetRange(topic, partition, fromOffset, untilOffset) def apply( - topicAndPartition: TopicAndPartition, + 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-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala deleted file mode 100644 index d66830cbacdee..0000000000000 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ /dev/null @@ -1,81 +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 - -import scala.util.Random - -import kafka.common.TopicAndPartition -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark.SparkFunSuite - -class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { - private val topic = "kcsuitetopic" + Random.nextInt(10000) - private val topicAndPartition = TopicAndPartition(topic, 0) - private var kc: KafkaCluster = null - - private var kafkaTestUtils: KafkaTestUtils = _ - - override def beforeAll() { - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) - kc = new KafkaCluster(Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress)) - } - - override def afterAll() { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } - - test("metadata apis") { - val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) - val leaderAddress = s"${leader._1}:${leader._2}" - assert(leaderAddress === kafkaTestUtils.brokerAddress, "didn't get leader") - - val parts = kc.getPartitions(Set(topic)).right.get - assert(parts(topicAndPartition), "didn't get partitions") - - val err = kc.getPartitions(Set(topic + "BAD")) - assert(err.isLeft, "getPartitions for a nonexistant topic should be an error") - } - - test("leader offset apis") { - val earliest = kc.getEarliestLeaderOffsets(Set(topicAndPartition)).right.get - assert(earliest(topicAndPartition).offset === 0, "didn't get earliest") - - val latest = kc.getLatestLeaderOffsets(Set(topicAndPartition)).right.get - assert(latest(topicAndPartition).offset === 1, "didn't get latest") - } - - test("consumer offset apis") { - val group = "kcsuitegroup" + Random.nextInt(10000) - - val offset = Random.nextInt(10000) - - val set = kc.setConsumerOffsets(group, Map(topicAndPartition -> offset)) - assert(set.isRight, "didn't set consumer offsets") - - val get = kc.getConsumerOffsets(group, Set(topicAndPartition)).right.get - assert(get(topicAndPartition) === offset, "didn't get consumer offsets") - } -} From f37717cd210367818be4fc6bd16557d94360b11e Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Mon, 7 Mar 2016 16:47:53 -0600 Subject: [PATCH 04/37] [SPARK-12177][Streaming][Kafka] WIP merge backpressure fixes from master --- .../kafka/DirectKafkaInputDStream.scala | 45 +++++++----- .../streaming/kafka/KafkaTestUtils.scala | 13 +++- .../kafka/DirectKafkaStreamSuite.scala | 68 ++++++++++++++++--- 3 files changed, 98 insertions(+), 28 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 407b0fe86fdae..e6ef4ac5dffc1 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -202,24 +202,33 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } private val maxRateLimitPerPartition: Int = context.sparkContext.getConf.getInt( - "spark.streaming.kafka.maxRatePerPartition", 0) - protected def maxMessagesPerPartition: Option[Long] = { + "spark.streaming.kafka.maxRatePerPartition", 0) + + protected[streaming] def maxMessagesPerPartition( + offsets: Map[TopicPartition, Long]): Option[Map[TopicPartition, 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 + + // calculate a per-partition rate limit based on current lag + val effectiveRateLimitPerPartition = estimatedRateLimit.filter(_ > 0) match { + case Some(rate) => + val lagPerPartition = offsets.map { case (tp, offset) => + tp -> Math.max(offset - currentOffsets.getOrElse(tp, offset), 0) + } + val totalLag = lagPerPartition.values.sum + + lagPerPartition.map { case (tp, lag) => + val backpressureRate = Math.round(lag / totalLag.toFloat * rate) + tp -> (if (maxRateLimitPerPartition > 0) { + Math.min(backpressureRate, maxRateLimitPerPartition)} else backpressureRate) } - }.getOrElse(maxRateLimitPerPartition) + case None => offsets.map { case (tp, offset) => tp -> maxRateLimitPerPartition } + } - if (effectiveRateLimitPerPartition > 0) { + if (effectiveRateLimitPerPartition.values.sum > 0) { val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 - Some((secsPerBatch * effectiveRateLimitPerPartition).toLong) + Some(effectiveRateLimitPerPartition.map { + case (tp, limit) => tp -> (secsPerBatch * limit).toLong + }) } else { None } @@ -238,9 +247,11 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( // limits the maximum number of messages per partition protected def clamp( offsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { - maxMessagesPerPartition.map { mmp => - offsets.map { case (tp, o) => - tp -> Math.min(currentOffsets(tp) + mmp, o) + + maxMessagesPerPartition(offsets).map { mmp => + mmp.map { case (tp, messages) => + val uo = offsets(tp) + tp -> Math.min(currentOffsets.getOrElse(tp, uo) + messages, uo) } }.getOrElse(offsets) } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index 2bc27b88ed7a1..602b77285e70a 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -150,10 +150,17 @@ private[kafka] class KafkaTestUtils extends Logging { } /** 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) + def createTopic(topic: String, partitions: Int): Unit = { + AdminUtils.createTopic(zkUtils, topic, partitions, 1) // wait until metadata is propagated - waitUntilMetadataIsPropagated(topic, 0) + (0 until partitions).foreach { p => + waitUntilMetadataIsPropagated(topic, p) + } + } + + /** Create a Kafka topic and wait until it is propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + createTopic(topic, 1) } /** Java-friendly function for sending messages to the Kafka broker */ diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 8398178e9b79b..b2c81d1534ee6 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -353,10 +353,38 @@ class DirectKafkaStreamSuite ssc.stop() } + test("maxMessagesPerPartition with backpressure disabled") { + val topic = "maxMessagesPerPartition" + val kafkaStream = getDirectKafkaStream(topic, None) + + val input = Map(TopicAndPartition(topic, 0) -> 50L, TopicAndPartition(topic, 1) -> 50L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + } + + test("maxMessagesPerPartition with no lag") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 100)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) + assert(kafkaStream.maxMessagesPerPartition(input).isEmpty) + } + + test("maxMessagesPerPartition respects max rate") { + val topic = "maxMessagesPerPartition" + val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 1000)) + val kafkaStream = getDirectKafkaStream(topic, rateController) + + val input = Map(TopicAndPartition(topic, 0) -> 1000L, TopicAndPartition(topic, 1) -> 1000L) + assert(kafkaStream.maxMessagesPerPartition(input).get == + Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + } + test("using rate controller") { val topic = "backpressure" - val topicPartition = TopicAndPartition(topic, 0) - kafkaTestUtils.createTopic(topic) + val topicPartitions = Set(TopicAndPartition(topic, 0), TopicAndPartition(topic, 1)) + kafkaTestUtils.createTopic(topic, 2) val kafkaParams = Map( "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "smallest" @@ -364,8 +392,8 @@ class DirectKafkaStreamSuite val batchIntervalMilliseconds = 100 val estimator = new ConstantEstimator(100) - val messageKeys = (1 to 200).map(_.toString) - val messages = messageKeys.map((_, 1)).toMap + val messages = Map("foo" -> 200) + kafkaTestUtils.sendMessages(topic, messages) val sparkConf = new SparkConf() // Safe, even with streaming, because we're using the direct API. @@ -380,11 +408,11 @@ class DirectKafkaStreamSuite val kafkaStream = withClue("Error creating direct stream") { val kc = new KafkaCluster(kafkaParams) val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - val m = kc.getEarliestLeaderOffsets(Set(topicPartition)) + val m = kc.getEarliestLeaderOffsets(topicPartitions) .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, kafkaParams, m, messageHandler) { + ssc, kafkaParams, m, messageHandler) { override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) } @@ -405,13 +433,12 @@ class DirectKafkaStreamSuite ssc.start() // Try different rate limits. - // Send data to Kafka and wait for arrays of data to appear matching the rate. + // 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. @@ -430,6 +457,25 @@ class DirectKafkaStreamSuite rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges }.toSeq.sortBy { _._1 } } + + private def getDirectKafkaStream(topic: String, mockRateController: Option[RateController]) = { + val batchIntervalMilliseconds = 100 + + val sparkConf = new SparkConf() + .setMaster("local[1]") + .setAppName(this.getClass.getSimpleName) + .set("spark.streaming.kafka.maxRatePerPartition", "100") + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) + + val earliestOffsets = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) + val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) + new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( + ssc, Map[String, String](), earliestOffsets, messageHandler) { + override protected[streaming] val rateController = mockRateController + } + } } object DirectKafkaStreamSuite { @@ -468,3 +514,9 @@ private[streaming] class ConstantEstimator(@volatile private var rate: Long) processingDelay: Long, schedulingDelay: Long): Option[Double] = Some(rate) } + +private[streaming] class ConstantRateController(id: Int, estimator: RateEstimator, rate: Long) + extends RateController(id, estimator) { + override def publish(rate: Long): Unit = () + override def getLatestRate(): Long = rate +} From e01daea67beccd14c1c6f77a3efd57282b42f1dd Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 8 Mar 2016 13:25:38 -0600 Subject: [PATCH 05/37] [SPARK-12177][Streaming][Kafka] WIP remove KafkaUtils until interface for new consumers is finished --- .../streaming/kafka/CachedKafkaConsumer.scala | 8 +- .../kafka/DirectKafkaInputDStream.scala | 30 +- .../spark/streaming/kafka/KafkaUtils.scala | 655 ------------------ 3 files changed, 31 insertions(+), 662 deletions(-) delete mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index 4ce52a6ca9c3b..c4c92157f30ef 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -22,7 +22,7 @@ import java.{ util => ju } import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord, KafkaConsumer } import org.apache.kafka.common.TopicPartition -import org.apache.spark.SparkConf +import org.apache.spark.{ Logging, SparkConf } /** Consumer of single topicpartition, intended for cached reuse. * Underlying consumer is not threadsafe, so neither is this, @@ -33,7 +33,7 @@ class CachedKafkaConsumer[K, V] private( val groupId: String, val topic: String, val partition: Int, - val kafkaParams: ju.Map[String, Object]) { + val kafkaParams: ju.Map[String, Object]) extends Logging { assert(groupId == kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG), "groupId used for cache key must match the groupId in kafkaParams") @@ -59,6 +59,7 @@ class CachedKafkaConsumer[K, V] private( var record = buffer.next() if (record.offset != offset) { + log.info(s"buffer miss for $groupId $topic $partition $offset") seek(offset) poll(timeout) record = buffer.next() @@ -80,7 +81,7 @@ class CachedKafkaConsumer[K, V] private( } private[kafka] -object CachedKafkaConsumer { +object CachedKafkaConsumer extends Logging { private case class CacheKey(groupId: String, topic: String, partition: Int) @@ -119,6 +120,7 @@ object CachedKafkaConsumer { val k = CacheKey(groupId, topic, partition) val v = cache.get(k) if (null == v) { + log.info(s"cache miss for $groupId $topic $partition") val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) cache.put(k, c) c diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index e6ef4ac5dffc1..2354e50623f56 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -51,7 +51,7 @@ import scala.collection.JavaConverters._ * NOT zookeeper servers, specified in host1:port1,host2:port2 form. */ -class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( +class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( _ssc: StreamingContext, val driverKafkaParams: ju.Map[String, Object], val executorKafkaParams: ju.Map[String, Object], @@ -156,6 +156,18 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( consumer.poll(pollTimeout) } + def seek(partition: TopicPartition, offset: Long): Unit = this.synchronized { + consumer.seek(partition, offset) + } + + def seekToBeginning(partitions: TopicPartition*): Unit = this.synchronized { + consumer.seekToBeginning(partitions: _*) + } + + def seekToEnd(partitions: TopicPartition*): Unit = this.synchronized { + consumer.seekToEnd(partitions: _*) + } + // TODO is there a better way to distinguish between // - want to use leader brokers (null map) // - don't care, use consistent executor (empty map) @@ -237,6 +249,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( protected var currentOffsets = Map[TopicPartition, Long]() protected def latestOffsets(): Map[TopicPartition, Long] = this.synchronized { + // TODO does this need a poll in order to maintain heartbeat / get topic updates val c = consumer c.seekToEnd() c.assignment().asScala.map { tp => @@ -289,9 +302,11 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( assert(partitionAssignment != Unassigned, "Must call subscribe or assign before starting") val c = consumer c.poll(pollTimeout) - currentOffsets = c.assignment().asScala.map { tp => - tp -> c.position(tp) - }.toMap + if (currentOffsets.isEmpty) { + currentOffsets = c.assignment().asScala.map { tp => + tp -> c.position(tp) + }.toMap + } } override def stop(): Unit = { @@ -356,4 +371,11 @@ object DirectKafkaInputDStream { /** Not yet assigned */ protected case object Unassigned extends PartitionAssignment + def apply[K: ClassTag, V: ClassTag]( + ssc: StreamingContext, + driverKafkaParams: ju.Map[String, Object], + executorKafkaParams: ju.Map[String, Object], + preferredHosts: ju.Map[TopicPartition, String]): DirectKafkaInputDStream[K, V] = { + new DirectKafkaInputDStream[K, V](ssc, driverKafkaParams, executorKafkaParams, preferredHosts) + } } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala deleted file mode 100644 index af8f1863ecea9..0000000000000 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ /dev/null @@ -1,655 +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 - -import java.io.OutputStream -import java.lang.{Integer => JInt, Long => JLong} -import java.util.{List => JList, Map => JMap, Set => JSet} - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import com.google.common.base.Charsets.UTF_8 -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} -import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} - -import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} -import org.apache.spark.api.java.function.{Function => JFunction} -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} -import org.apache.spark.streaming.util.WriteAheadLogUtils - -object KafkaUtils { - /** get leaders for the given offset ranges, or throw an exception */ - private def leadersForRanges( - kc: KafkaCluster, - offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { - val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet - val leaders = kc.findLeaders(topics) - KafkaCluster.checkErrors(leaders) - } - - /** 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.getEarliestLeaderOffsets(topics).right - high <- kc.getLatestLeaderOffsets(topics).right - } yield { - offsetRanges.filterNot { o => - low(o.topicAndPartition).offset <= o.fromOffset && - o.untilOffset <= high(o.topicAndPartition).offset - } - } - val badRanges = KafkaCluster.checkErrors(result) - if (!badRanges.isEmpty) { - throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) - } - } - - private[kafka] def getFromOffsets( - kc: KafkaCluster, - kafkaParams: Map[String, String], - topics: Set[String] - ): Map[TopicAndPartition, Long] = { - val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase) - val result = for { - topicPartitions <- kc.getPartitions(topics).right - leaderOffsets <- (if (reset == Some("smallest")) { - kc.getEarliestLeaderOffsets(topicPartitions) - } else { - kc.getLatestLeaderOffsets(topicPartitions) - }).right - } yield { - leaderOffsets.map { case (tp, lo) => - (tp, lo.offset) - } - } - KafkaCluster.checkErrors(result) - } - - /** - * Create a RDD from Kafka using offset ranges for each topic and partition. - * - * @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 - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return RDD of (Kafka message key, Kafka message value) - */ - def createRDD[ - K: ClassTag, - V: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange] - ): RDD[(K, V)] = sc.withScope { - val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val leaders = leadersForRanges(kc, offsetRanges) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, (K, V)](sc, kafkaParams, offsetRanges, leaders, 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 leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, - * in which case leaders will be looked up on the driver. - * @param messageHandler Function for translating each message and metadata into the desired type - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return RDD of R - */ - def createRDD[ - K: ClassTag, - V: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange], - leaders: Map[TopicAndPartition, Broker], - messageHandler: MessageAndMetadata[K, V] => R - ): RDD[R] = sc.withScope { - val kc = new KafkaCluster(kafkaParams) - val leaderMap = if (leaders.isEmpty) { - leadersForRanges(kc, offsetRanges) - } else { - // This could be avoided by refactoring KafkaRDD.leaders and KafkaCluster to use Broker - leaders.map { - case (tp: TopicAndPartition, Broker(host, port)) => (tp, (host, port)) - } - } - val cleanedHandler = sc.clean(messageHandler) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, cleanedHandler) - } - - /** - * Create a RDD from Kafka using offset ranges for each topic and partition. - * - * @param jsc JavaSparkContext 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 keyClass type of Kafka message key - * @param valueClass type of Kafka message value - * @param keyDecoderClass type of Kafka message key decoder - * @param valueDecoderClass type of Kafka message value decoder - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return RDD of (Kafka message key, Kafka message value) - */ - def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - 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) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - 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 "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 leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, - * in which case leaders will be looked up on the driver. - * @param messageHandler Function for translating each message and metadata into the desired type - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return RDD of R - */ - def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - recordClass: Class[R], - kafkaParams: JMap[String, String], - offsetRanges: Array[OffsetRange], - leaders: JMap[TopicAndPartition, Broker], - messageHandler: JFunction[MessageAndMetadata[K, V], R] - ): JavaRDD[R] = jsc.sc.withScope { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) - val leaderMap = Map(leaders.asScala.toSeq: _*) - createRDD[K, V, KD, VD, R]( - jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, leaderMap, 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.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 - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return DStream of R - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag, - R: ClassTag] ( - ssc: StreamingContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - messageHandler: MessageAndMetadata[K, V] => R - ): InputDStream[R] = { - val cleanedHandler = ssc.sc.clean(messageHandler) - new DirectKafkaInputDStream[K, V, KD, VD, R]( - ssc, kafkaParams, fromOffsets, 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.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 - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return DStream of (Kafka message key, Kafka message value) - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag] ( - ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Set[String] - ): InputDStream[(K, V)] = { - val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val fromOffsets = getFromOffsets(kc, kafkaParams, topics) - new DirectKafkaInputDStream[K, V, KD, VD, (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.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 keyDecoderClass Class of the key decoder - * @param valueDecoderClass Class of the value decoder - * @param recordClass Class of the records in DStream - * @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 - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return DStream of R - */ - def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - recordClass: Class[R], - kafkaParams: JMap[String, String], - fromOffsets: JMap[TopicAndPartition, JLong], - messageHandler: JFunction[MessageAndMetadata[K, V], R] - ): JavaInputDStream[R] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) - val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) - createDirectStream[K, V, KD, VD, 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.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 keyDecoderClass Class of the key decoder - * @param valueDecoderClass Class type of the value decoder - * @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 - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return DStream of (Kafka message key, Kafka message value) - */ - def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - kafkaParams: JMap[String, String], - topics: JSet[String] - ): JavaPairInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - createDirectStream[K, V, KD, VD]( - jssc.ssc, - Map(kafkaParams.asScala.toSeq: _*), - Set(topics.asScala.toSeq: _*) - ) - } -} - -/** - * 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], - leaders: JMap[TopicAndPartition, Broker]): JavaRDD[(Array[Byte], Array[Byte])] = { - val messageHandler = - (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) - new JavaRDD(createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler)) - } - - def createRDDWithMessageHandler( - jsc: JavaSparkContext, - kafkaParams: JMap[String, String], - offsetRanges: JList[OffsetRange], - leaders: JMap[TopicAndPartition, Broker]): JavaRDD[Array[Byte]] = { - val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => - new PythonMessageAndMetadata( - mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) - val rdd = createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler). - mapPartitions(picklerIterator) - new JavaRDD(rdd) - } - - private def createRDD[V: ClassTag]( - jsc: JavaSparkContext, - kafkaParams: JMap[String, String], - offsetRanges: JList[OffsetRange], - leaders: JMap[TopicAndPartition, Broker], - messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): RDD[V] = { - KafkaUtils.createRDD[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( - jsc.sc, - kafkaParams.asScala.toMap, - offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), - leaders.asScala.toMap, - messageHandler - ) - } - - def createDirectStreamWithoutMessageHandler( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JLong]): JavaDStream[(Array[Byte], Array[Byte])] = { - val messageHandler = - (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) - new JavaDStream(createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler)) - } - - def createDirectStreamWithMessageHandler( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JLong]): JavaDStream[Array[Byte]] = { - val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => - new PythonMessageAndMetadata(mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) - 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[TopicAndPartition, JLong], - messageHandler: MessageAndMetadata[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 { - val kc = new KafkaCluster(Map(kafkaParams.asScala.toSeq: _*)) - KafkaUtils.getFromOffsets( - kc, Map(kafkaParams.asScala.toSeq: _*), Set(topics.asScala.toSeq: _*)) - } - - KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, 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): TopicAndPartition = - TopicAndPartition(topic, partition) - - def createBroker(host: String, port: JInt): Broker = Broker(host, port) - - 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 { - 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) - } - } - } -} From 20b59e1970753f242cdde01ac9a6f131833cd6d4 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 8 Mar 2016 14:45:50 -0600 Subject: [PATCH 06/37] [SPARK-12177][Streaming][Kafka] fix handling of kafkaParams --- .../kafka/DirectKafkaInputDStream.scala | 28 +++++++++++++++++-- 1 file changed, 25 insertions(+), 3 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 2354e50623f56..d961fb99824c9 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -69,7 +69,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( assert(false == driverKafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + " must be set to false for driver kafka params, else offsets may commit before processing") @transient private var kc: KafkaConsumer[K, V] = null @@ -351,7 +351,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( } } -object DirectKafkaInputDStream { +object DirectKafkaInputDStream extends Logging { protected val defaultListener = "org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener" /** There are several different ways of specifying partition assignment, @@ -376,6 +376,28 @@ object DirectKafkaInputDStream { driverKafkaParams: ju.Map[String, Object], executorKafkaParams: ju.Map[String, Object], preferredHosts: ju.Map[TopicPartition, String]): DirectKafkaInputDStream[K, V] = { - new DirectKafkaInputDStream[K, V](ssc, driverKafkaParams, executorKafkaParams, preferredHosts) + val dkp = new ju.HashMap[String, Object](driverKafkaParams) + val ekp = new ju.HashMap[String, Object](executorKafkaParams) + + log.warn(s"overriding ${ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG} to 1 for driver") + dkp.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 1: Integer) + + log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for driver") + dkp.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + + log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") + ekp.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + + log.warn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") + ekp.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // this probably doesnt matter since executors are manually assigned partitions, but just in case + if (null != ekp.get(ConsumerConfig.GROUP_ID_CONFIG)) { + val id = ekp.get(ConsumerConfig.GROUP_ID_CONFIG) + "-executor" + log.warn(s"overriding ${ConsumerConfig.GROUP_ID_CONFIG} to ${id} for executor") + ekp.put(ConsumerConfig.GROUP_ID_CONFIG, id) + } + + new DirectKafkaInputDStream[K, V](ssc, dkp, ekp, preferredHosts) } } From b55a3d41973d6838b1cb45ec2aa45d8bc6585cd3 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 8 Mar 2016 17:01:29 -0600 Subject: [PATCH 07/37] [SPARK-12177][Streaming][Kafka] better handling of dynamic topic subscriptions --- .../kafka/DirectKafkaInputDStream.scala | 36 +++++++++++-------- 1 file changed, 21 insertions(+), 15 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index d961fb99824c9..a54291f48e4d5 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -89,13 +89,20 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( .asInstanceOf[ConsumerRebalanceListener] private def assignPartitions(pa: PartitionAssignment): Unit = this.synchronized { + val reset = driverKafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).asInstanceOf[String] + .toLowerCase + val resetMsg = "Dynamic topic subscriptions won't work well unless " + + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is set" + // using kc directly because consumer() calls this method pa match { case Assigned(partitions) => kc.assign(partitions) case Subscribed(topics, className) => + assert(reset == "earliest" || reset == "latest", resetMsg) kc.subscribe(topics, listenerFor(className)) case PatternSubscribed(pattern, className) => + assert(reset == "earliest" || reset == "latest", resetMsg) kc.subscribe(pattern, listenerFor(className)) case Unassigned => } @@ -146,14 +153,9 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( consumer.partitionsFor(topic) } - private val pollTimeout = - context.sparkContext.getConf.getLong("spark.streaming.kafka.consumer.poll.ms", 10) - /** Necessary to fetch metadata and update subscriptions, won't actually return useful data */ - def poll(): Unit = poll(pollTimeout) - /** Necessary to fetch metadata and update subscriptions, won't actually return useful data */ def poll(timeout: Long): Unit = this.synchronized { - consumer.poll(pollTimeout) + consumer.poll(timeout) } def seek(partition: TopicPartition, offset: Long): Unit = this.synchronized { @@ -224,7 +226,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( val effectiveRateLimitPerPartition = estimatedRateLimit.filter(_ > 0) match { case Some(rate) => val lagPerPartition = offsets.map { case (tp, offset) => - tp -> Math.max(offset - currentOffsets.getOrElse(tp, offset), 0) + tp -> Math.max(offset - currentOffsets(tp), 0) } val totalLag = lagPerPartition.values.sum @@ -249,12 +251,16 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( protected var currentOffsets = Map[TopicPartition, Long]() protected def latestOffsets(): Map[TopicPartition, Long] = this.synchronized { - // TODO does this need a poll in order to maintain heartbeat / get topic updates val c = consumer + c.poll(0) + val parts = c.assignment().asScala + if (!partitionAssignment.isInstanceOf[Assigned]) { + // make sure new partitions are reflected in currentOffsets + val newPartitions = parts.diff(currentOffsets.keySet) + currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap + } c.seekToEnd() - c.assignment().asScala.map { tp => - tp -> c.position(tp) - }.toMap + parts.map(tp => tp -> c.position(tp)).toMap } // limits the maximum number of messages per partition @@ -264,7 +270,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( maxMessagesPerPartition(offsets).map { mmp => mmp.map { case (tp, messages) => val uo = offsets(tp) - tp -> Math.min(currentOffsets.getOrElse(tp, uo) + messages, uo) + tp -> Math.min(currentOffsets(tp) + messages, uo) } }.getOrElse(offsets) } @@ -272,7 +278,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( override def compute(validTime: Time): Option[KafkaRDD[K, V]] = { val untilOffsets = clamp(latestOffsets()) val offsetRanges = untilOffsets.map { case (tp, uo) => - val fo = currentOffsets.getOrElse(tp, uo) + val fo = currentOffsets(tp) OffsetRange(tp.topic, tp.partition, fo, uo) } @@ -301,7 +307,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( override def start(): Unit = this.synchronized { assert(partitionAssignment != Unassigned, "Must call subscribe or assign before starting") val c = consumer - c.poll(pollTimeout) + c.poll(0) if (currentOffsets.isEmpty) { currentOffsets = c.assignment().asScala.map { tp => tp -> c.position(tp) @@ -332,7 +338,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( override def cleanup(time: Time) { } override def restore() { - poll() + poll(0) batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") From d2949f802489f29a2107c2ee441a89d394ed7769 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 9 Mar 2016 11:48:42 -0600 Subject: [PATCH 08/37] [SPARK-12177][Streaming][Kafka] WIP cached consumers basically working, but dont handle recalculating the same RDD efficiently --- .../streaming/kafka/CachedKafkaConsumer.scala | 41 ++++++++++---- .../kafka/DirectKafkaInputDStream.scala | 56 +++++++++---------- .../spark/streaming/kafka/KafkaRDD.scala | 39 ++++++++++--- 3 files changed, 88 insertions(+), 48 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index c4c92157f30ef..b597dd5eea0e0 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -49,33 +49,50 @@ class CachedKafkaConsumer[K, V] private( } protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + protected var nextOffset = -2L /** Get the record for the given offset, waiting up to timeout ms if IO is necessary. * Sequential forward access will use buffers, but random access will be horribly inefficient. */ def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = { - if (!buffer.hasNext) { poll(timeout) } + log.debug(s"get $groupId $topic $partition nextOffset $nextOffset requested $offset") + if (offset != nextOffset) { + log.info(s"initial fetch for $groupId $topic $partition $offset") + buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + seek(offset) + poll(timeout) + } + if (!buffer.hasNext()) { poll(timeout) } + assert(buffer.hasNext(), + s"failed to get records for $groupId $topic $partition $offset after polling for $timeout") var record = buffer.next() if (record.offset != offset) { log.info(s"buffer miss for $groupId $topic $partition $offset") seek(offset) poll(timeout) + assert(buffer.hasNext(), + s"failed to get records for $groupId $topic $partition $offset after polling for $timeout") record = buffer.next() assert(record.offset == offset, - s"Failed to get offset $offset after seeking to it") + s"Got wrong record for $groupId $topic $partition even after seeking to offset $offset") } + nextOffset = offset + 1 record } private def seek(offset: Long): Unit = { + log.debug(s"seeking to $topicPartition $offset") consumer.seek(topicPartition, offset) } private def poll(timeout: Long): Unit = { - buffer = consumer.poll(timeout).records(topicPartition).iterator + val p = consumer.poll(timeout) + val r = p.records(topicPartition) + log.debug(s"polled ${p.partitions()} ${r.size}") + buffer = r.iterator } } @@ -89,15 +106,17 @@ object CachedKafkaConsumer extends Logging { private var cache: ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]] = null /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ - def init(conf: SparkConf): Unit = CachedKafkaConsumer.synchronized { + def init( + initialCapacity: Int, + maxCapacity: Int, + loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { if (null == cache) { - val initial = conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) - val max = conf.getInt("spark.streaming.kafka.consumer.cache.maxCapacity", 64) - val load = conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat - cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]](initial, load, true) { + log.info(s"initializing cache $initialCapacity $maxCapacity $loadFactor") + cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( + initialCapacity, loadFactor, true) { override def removeEldestEntry( entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer[_, _]]): Boolean = { - if (this.size > max) { + if (this.size > maxCapacity) { entry.getValue.consumer.close() true } else { @@ -120,7 +139,8 @@ object CachedKafkaConsumer extends Logging { val k = CacheKey(groupId, topic, partition) val v = cache.get(k) if (null == v) { - log.info(s"cache miss for $groupId $topic $partition") + log.info(s"cache miss for $k") + log.debug(cache.keySet.toString) val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) cache.put(k, c) c @@ -134,6 +154,7 @@ object CachedKafkaConsumer extends Logging { def remove(groupId: String, topic: String, partition: Int): Unit = CachedKafkaConsumer.synchronized { val k = CacheKey(groupId, topic, partition) + log.info(s"removing $k from cache") val v = cache.get(k) if (null != v) { v.consumer.close() diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index a54291f48e4d5..174fac812ea0c 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -51,7 +51,7 @@ import scala.collection.JavaConverters._ * NOT zookeeper servers, specified in host1:port1,host2:port2 form. */ -class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( +class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[kafka] ( _ssc: StreamingContext, val driverKafkaParams: ju.Map[String, Object], val executorKafkaParams: ju.Map[String, Object], @@ -62,11 +62,6 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( PartitionAssignment, Assigned, Subscribed, PatternSubscribed, Unassigned } - assert(1 == - driverKafkaParams.get(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG).asInstanceOf[Int], - ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG + - " must be set to 1 for driver kafka params, because the driver should not consume messages") - assert(false == driverKafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + @@ -83,26 +78,33 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( } consumer() + // XXX TODO listeners arent much use if they dont have a reference to the consumer private def listenerFor(className: String): ConsumerRebalanceListener = Class.forName(className) .newInstance() .asInstanceOf[ConsumerRebalanceListener] private def assignPartitions(pa: PartitionAssignment): Unit = this.synchronized { + import DirectKafkaInputDStream.defaultListener + val reset = driverKafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).asInstanceOf[String] .toLowerCase - val resetMsg = "Dynamic topic subscriptions won't work well unless " + - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is set" + val resetMsg = "Dynamic topic subscriptions may not work well unless " + + ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is set or a rebalance listener is configured" // using kc directly because consumer() calls this method pa match { case Assigned(partitions) => kc.assign(partitions) case Subscribed(topics, className) => - assert(reset == "earliest" || reset == "latest", resetMsg) + if (!(reset == "earliest" || reset == "latest" || className != defaultListener)) { + log.warn(resetMsg) + } kc.subscribe(topics, listenerFor(className)) case PatternSubscribed(pattern, className) => - assert(reset == "earliest" || reset == "latest", resetMsg) + if (!(reset == "earliest" || reset == "latest" || className != defaultListener)) { + log.warn(resetMsg) + } kc.subscribe(pattern, listenerFor(className)) case Unassigned => } @@ -342,7 +344,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private ( batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") - generatedRDDs += t -> KafkaRDD[K, V]( + generatedRDDs += t -> new KafkaRDD[K, V]( context.sparkContext, executorKafkaParams, b.map(OffsetRange(_)), getPreferredHosts) } } @@ -377,6 +379,15 @@ object DirectKafkaInputDStream extends Logging { /** Not yet assigned */ protected case object Unassigned extends PartitionAssignment + private def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { + // wouldn't this be nice to have... 0.10 has it, but with a minimum of 1 + // log.warn(s"overriding ${ConsumerConfig.MAX_POLL_RECORDS_CONFIG} to 0 for driver") + // kafkaParams.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 0: Integer) + + log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for driver") + kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + } + def apply[K: ClassTag, V: ClassTag]( ssc: StreamingContext, driverKafkaParams: ju.Map[String, Object], @@ -384,26 +395,11 @@ object DirectKafkaInputDStream extends Logging { preferredHosts: ju.Map[TopicPartition, String]): DirectKafkaInputDStream[K, V] = { val dkp = new ju.HashMap[String, Object](driverKafkaParams) val ekp = new ju.HashMap[String, Object](executorKafkaParams) + val ph = new ju.HashMap[TopicPartition, String](preferredHosts) - log.warn(s"overriding ${ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG} to 1 for driver") - dkp.put(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, 1: Integer) - - log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for driver") - dkp.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) - - log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") - ekp.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) - - log.warn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") - ekp.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") - - // this probably doesnt matter since executors are manually assigned partitions, but just in case - if (null != ekp.get(ConsumerConfig.GROUP_ID_CONFIG)) { - val id = ekp.get(ConsumerConfig.GROUP_ID_CONFIG) + "-executor" - log.warn(s"overriding ${ConsumerConfig.GROUP_ID_CONFIG} to ${id} for executor") - ekp.put(ConsumerConfig.GROUP_ID_CONFIG, id) - } + fixKafkaParams(dkp) + KafkaRDD.fixKafkaParams(ekp) - new DirectKafkaInputDStream[K, V](ssc, dkp, ekp, preferredHosts) + new DirectKafkaInputDStream[K, V](ssc, dkp, ekp, ph) } } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 1613f9cc5a2be..9c28847ca3c28 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -42,7 +42,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation class KafkaRDD[ K: ClassTag, - V: ClassTag] private ( + V: ClassTag] private[kafka] ( sc: SparkContext, val kafkaParams: ju.Map[String, Object], val offsetRanges: Array[OffsetRange], @@ -59,6 +59,14 @@ class KafkaRDD[ ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " must be set to false for executor kafka params, else offsets may commit before processing") + private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", 64) + private val cacheInitialCapacity = + conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) + private val cacheMaxCapacity = + conf.getInt("spark.streaming.kafka.consumer.cache.maxCapacity", 64) + private val cacheLoadFactor = + conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat + override def getPartitions: Array[Partition] = { offsetRanges.zipWithIndex.map { case (o, i) => new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) @@ -162,12 +170,10 @@ class KafkaRDD[ val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] - val pollTimeout = sparkContext.getConf.getLong("spark.streaming.kafka.consumer.poll.ms", 10) - val consumer = { - CachedKafkaConsumer.init(sparkContext.getConf) - if (context.attemptNumber > 0) { - // just in case the prior attempt failure was cache related + CachedKafkaConsumer.init(cacheInitialCapacity, cacheMaxCapacity, cacheLoadFactor) + if (context.attemptNumber > 1) { + // just in case the prior attempt failures were cache related CachedKafkaConsumer.remove(groupId, part.topic, part.partition) } CachedKafkaConsumer.get[K, V](groupId, part.topic, part.partition, kafkaParams) @@ -186,7 +192,21 @@ class KafkaRDD[ } } -object KafkaRDD { +object KafkaRDD extends Logging { + private[kafka] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { + log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") + kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + + log.warn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") + kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // this probably doesnt matter since executors are manually assigned partitions, but just in case + if (null != kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)) { + val id = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + "-executor" + log.warn(s"overriding ${ConsumerConfig.GROUP_ID_CONFIG} to ${id} for executor") + kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, id) + } + } def apply[ K: ClassTag, @@ -196,7 +216,10 @@ object KafkaRDD { offsetRanges: Array[OffsetRange], preferredHosts: ju.Map[TopicPartition, String] ): KafkaRDD[K, V] = { + val kp = new ju.HashMap[String, Object](kafkaParams) + val osr = offsetRanges.clone() + val ph = new ju.HashMap[TopicPartition, String](preferredHosts) - new KafkaRDD[K, V](sc, kafkaParams, offsetRanges, preferredHosts) + new KafkaRDD[K, V](sc, kp, osr, ph) } } From 47209c02c6c095e58debfd10b182ec74ef979af8 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 9 Mar 2016 11:57:26 -0600 Subject: [PATCH 09/37] [SPARK-12177][Streaming][Kafka] WIP remove unused class --- .../apache/spark/streaming/kafka/Broker.scala | 66 ------------------- 1 file changed, 66 deletions(-) delete mode 100644 external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala deleted file mode 100644 index 9159051ba06e4..0000000000000 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala +++ /dev/null @@ -1,66 +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 - -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)) - } - } -} From dea8ec08d54050c27891fb39bba0d183aede55ab Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Thu, 10 Mar 2016 14:02:31 -0600 Subject: [PATCH 10/37] [SPARK-12177][Streaming][Kafka] WIP add java convenience constructors, fix tests --- .../streaming/kafka/CachedKafkaConsumer.scala | 2 + .../kafka/DirectKafkaInputDStream.scala | 67 +++++-- .../spark/streaming/kafka/KafkaRDD.scala | 65 ++++--- .../kafka/JavaDirectKafkaStreamSuite.java | 95 ++++++---- .../streaming/kafka/JavaKafkaRDDSuite.java | 79 +++----- .../kafka/DirectKafkaStreamSuite.scala | 168 ++++++++++-------- .../spark/streaming/kafka/KafkaRDDSuite.scala | 104 ++++------- 7 files changed, 303 insertions(+), 277 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index b597dd5eea0e0..11b0eda7da70c 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -48,6 +48,8 @@ class CachedKafkaConsumer[K, V] private( c } + // TODO if the buffer was kept around as a random-access structure, + // could possibly optimize re-calculating of an RDD in the same batch protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator protected var nextOffset = -2L diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 174fac812ea0c..1abff4e6a3843 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -51,7 +51,7 @@ import scala.collection.JavaConverters._ * NOT zookeeper servers, specified in host1:port1,host2:port2 form. */ -class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[kafka] ( +class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( _ssc: StreamingContext, val driverKafkaParams: ju.Map[String, Object], val executorKafkaParams: ju.Map[String, Object], @@ -155,11 +155,19 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[kafka] ( consumer.partitionsFor(topic) } - /** Necessary to fetch metadata and update subscriptions, won't actually return useful data */ + /** Necessary to fetch metadata and update subscriptions, + * driver shouldn't be reading messages, so don't call this with non-zero timeout */ def poll(timeout: Long): Unit = this.synchronized { consumer.poll(timeout) } + /** Get the offset of the next record that will be fetched + * (if a record with that offset exists). + */ + def position(partition: TopicPartition): Long = this.synchronized { + consumer.position(partition) + } + def seek(partition: TopicPartition, offset: Long): Unit = this.synchronized { consumer.seek(partition, offset) } @@ -220,8 +228,8 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[kafka] ( private val maxRateLimitPerPartition: Int = context.sparkContext.getConf.getInt( "spark.streaming.kafka.maxRatePerPartition", 0) - protected[streaming] def maxMessagesPerPartition( - offsets: Map[TopicPartition, Long]): Option[Map[TopicPartition, Long]] = { + protected[streaming] def maxMessagesPerPartition( + offsets: Map[TopicPartition, Long]): Option[Map[TopicPartition, Long]] = { val estimatedRateLimit = rateController.map(_.getLatestRate().toInt) // calculate a per-partition rate limit based on current lag @@ -360,6 +368,8 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[kafka] ( } object DirectKafkaInputDStream extends Logging { + import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } + protected val defaultListener = "org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener" /** There are several different ways of specifying partition assignment, @@ -379,27 +389,60 @@ object DirectKafkaInputDStream extends Logging { /** Not yet assigned */ protected case object Unassigned extends PartitionAssignment - private def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { + private def fixKafkaParams( + dkp: ju.HashMap[String, Object], + ekp: ju.HashMap[String, Object] + ): Unit = { // wouldn't this be nice to have... 0.10 has it, but with a minimum of 1 // log.warn(s"overriding ${ConsumerConfig.MAX_POLL_RECORDS_CONFIG} to 0 for driver") - // kafkaParams.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 0: Integer) + // dkp.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 0: Integer) log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for driver") - kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + dkp.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + + // this probably doesnt matter since executors are manually assigned partitions, but just in case + if (null != ekp.get(ConsumerConfig.GROUP_ID_CONFIG) && + ekp.get(ConsumerConfig.GROUP_ID_CONFIG) == + dkp.get(ConsumerConfig.GROUP_ID_CONFIG) + ) { + val id = ekp.get(ConsumerConfig.GROUP_ID_CONFIG) + "-executor" + log.warn(s"overriding ${ConsumerConfig.GROUP_ID_CONFIG} to ${id} for executor") + ekp.put(ConsumerConfig.GROUP_ID_CONFIG, id) + } + } def apply[K: ClassTag, V: ClassTag]( - ssc: StreamingContext, - driverKafkaParams: ju.Map[String, Object], - executorKafkaParams: ju.Map[String, Object], - preferredHosts: ju.Map[TopicPartition, String]): DirectKafkaInputDStream[K, V] = { + ssc: StreamingContext, + driverKafkaParams: ju.Map[String, Object], + executorKafkaParams: ju.Map[String, Object], + preferredHosts: ju.Map[TopicPartition, String] + ): DirectKafkaInputDStream[K, V] = { val dkp = new ju.HashMap[String, Object](driverKafkaParams) val ekp = new ju.HashMap[String, Object](executorKafkaParams) val ph = new ju.HashMap[TopicPartition, String](preferredHosts) - fixKafkaParams(dkp) + fixKafkaParams(dkp, ekp) KafkaRDD.fixKafkaParams(ekp) new DirectKafkaInputDStream[K, V](ssc, dkp, ekp, ph) } + + def create[K, V]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + driverKafkaParams: ju.Map[String, Object], + executorKafkaParams: ju.Map[String, Object], + preferredHosts: ju.Map[TopicPartition, String] + ): JavaInputDStream[ConsumerRecord[K, V]] = { + + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + + new JavaInputDStream( + DirectKafkaInputDStream[K, V]( + jssc.ssc, driverKafkaParams, executorKafkaParams, preferredHosts)) + } + } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 9c28847ca3c28..23fe50d689f07 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -42,7 +42,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation class KafkaRDD[ K: ClassTag, - V: ClassTag] private[kafka] ( + V: ClassTag] private[spark] ( sc: SparkContext, val kafkaParams: ju.Map[String, Object], val offsetRanges: Array[OffsetRange], @@ -59,7 +59,8 @@ class KafkaRDD[ ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " must be set to false for executor kafka params, else offsets may commit before processing") - private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", 64) + // TODO is it necessary to have separate configs for initial poll time vs ongoing poll time? + private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", 256) private val cacheInitialCapacity = conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) private val cacheMaxCapacity = @@ -115,16 +116,11 @@ class KafkaRDD[ buf.toArray } - // TODO is there a better way to get executors - @transient private var sortedExecutors: Array[ExecutorCacheTaskLocation] = null private def executors(): Array[ExecutorCacheTaskLocation] = { - if (null == sortedExecutors) { - val bm = sparkContext.env.blockManager - sortedExecutors = bm.master.getPeers(bm.blockManagerId).toArray - .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) - .sortWith((a, b) => a.host > b.host || a.executorId > b.executorId) - } - sortedExecutors + val bm = sparkContext.env.blockManager + bm.master.getPeers(bm.blockManagerId).toArray + .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) + .sortWith((a, b) => a.host > b.host || a.executorId > b.executorId) } // non-negative modulus, from java 8 math @@ -138,10 +134,14 @@ class KafkaRDD[ val prefHost = preferredHosts.get(tp) val prefExecs = if (null == prefHost) allExecs else allExecs.filter(_.host == prefHost) val execs = if (prefExecs.isEmpty) allExecs else prefExecs - val index = floorMod(tp.hashCode, execs.length) - val chosen = execs(index) + if (execs.isEmpty) { + Seq() + } else { + val index = this.floorMod(tp.hashCode, execs.length) + val chosen = execs(index) - Seq(chosen.toString) + Seq(chosen.toString) + } } private def errBeginAfterEnd(part: KafkaRDDPartition): String = @@ -193,33 +193,42 @@ class KafkaRDD[ } object KafkaRDD extends Logging { + import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } + private[kafka] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) log.warn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") - - // this probably doesnt matter since executors are manually assigned partitions, but just in case - if (null != kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG)) { - val id = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + "-executor" - log.warn(s"overriding ${ConsumerConfig.GROUP_ID_CONFIG} to ${id} for executor") - kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, id) - } } - def apply[ - K: ClassTag, - V: ClassTag]( - sc: SparkContext, - kafkaParams: ju.Map[String, Object], - offsetRanges: Array[OffsetRange], - preferredHosts: ju.Map[TopicPartition, String] + def apply[K: ClassTag, V: ClassTag]( + sc: SparkContext, + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + preferredHosts: ju.Map[TopicPartition, String] ): KafkaRDD[K, V] = { val kp = new ju.HashMap[String, Object](kafkaParams) + fixKafkaParams(kp) val osr = offsetRanges.clone() val ph = new ju.HashMap[TopicPartition, String](preferredHosts) new KafkaRDD[K, V](sc, kp, osr, ph) } + + def create[K, V]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + preferredHosts: ju.Map[TopicPartition, String] + ): JavaRDD[ConsumerRecord[K, V]] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + + new JavaRDD(KafkaRDD[K, V](jsc.sc, kafkaParams, offsetRanges, preferredHosts)) + } + } diff --git a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index 4891e4f4a17bc..22c2060e48eeb 100644 --- a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -21,11 +21,10 @@ import java.util.*; import java.util.concurrent.atomic.AtomicReference; -import scala.Tuple2; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.clients.consumer.ConsumerRecord; -import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -38,6 +37,7 @@ import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaInputDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; public class JavaDirectKafkaStreamSuite implements Serializable { @@ -80,23 +80,36 @@ public void testKafkaStream() throws InterruptedException { sent.addAll(Arrays.asList(topic1data)); sent.addAll(Arrays.asList(topic2data)); - Map kafkaParams = new HashMap<>(); - kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); - kafkaParams.put("auto.offset.reset", "smallest"); + Random random = new Random(); - JavaDStream stream1 = KafkaUtils.createDirectStream( + Map kafkaParams = new HashMap<>(); + kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); + kafkaParams.put("key.deserializer", StringDeserializer.class); + kafkaParams.put("value.deserializer", StringDeserializer.class); + kafkaParams.put("auto.offset.reset", "earliest"); + kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); + + Map preferredHosts = new HashMap<>(); + + JavaInputDStream istream1 = DirectKafkaInputDStream.create( ssc, String.class, String.class, - StringDecoder.class, - StringDecoder.class, kafkaParams, - topicToSet(topic1) - ).transformToPair( + kafkaParams, + preferredHosts + ); + + ((DirectKafkaInputDStream) istream1.inputDStream()).subscribe(Arrays.asList(topic1)); + + JavaDStream stream1 = istream1.transform( // Make sure you can get offset ranges from the rdd - new Function, JavaPairRDD>() { + new Function>, + JavaRDD>>() { @Override - public JavaPairRDD call(JavaPairRDD rdd) { + public JavaRDD> call( + JavaRDD> rdd + ) { OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); offsetRanges.set(offsets); Assert.assertEquals(topic1, offsets[0].topic()); @@ -104,30 +117,50 @@ public JavaPairRDD call(JavaPairRDD rdd) { } } ).map( - new Function, String>() { + new Function, String>() { @Override - public String call(Tuple2 kv) { - return kv._2(); + public String call(ConsumerRecord r) { + return r.value(); } } ); - JavaDStream stream2 = KafkaUtils.createDirectStream( + kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); + + JavaInputDStream istream2 = DirectKafkaInputDStream.create( ssc, String.class, String.class, - StringDecoder.class, - StringDecoder.class, - String.class, kafkaParams, - topicOffsetToMap(topic2, 0L), - new Function, String>() { + kafkaParams, + preferredHosts + ); + + ((DirectKafkaInputDStream) istream2.inputDStream()).subscribe(Arrays.asList(topic2)); + + JavaDStream stream2 = istream2.transform( + // Make sure you can get offset ranges from the rdd + new Function>, + JavaRDD>>() { @Override - public String call(MessageAndMetadata msgAndMd) { - return msgAndMd.message(); + public JavaRDD> call( + JavaRDD> rdd + ) { + OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); + offsetRanges.set(offsets); + Assert.assertEquals(topic2, offsets[0].topic()); + return rdd; + } + } + ).map( + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); } } ); + JavaDStream unifiedStream = stream1.union(stream2); final Set result = Collections.synchronizedSet(new HashSet()); @@ -154,18 +187,6 @@ public void call(JavaRDD rdd) { ssc.stop(); } - private static Set topicToSet(String topic) { - Set topicSet = new HashSet<>(); - topicSet.add(topic); - return topicSet; - } - - private static Map topicOffsetToMap(String topic, Long offsetToStart) { - Map 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); diff --git a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index afcc6cfccd39a..6e02889c6d67b 100644 --- a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -21,11 +21,9 @@ import java.util.HashMap; import java.util.Map; -import scala.Tuple2; - -import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; +import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.clients.consumer.ConsumerRecord; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -70,81 +68,54 @@ public void testKafkaRDD() throws InterruptedException { createTopicAndSendData(topic1); createTopicAndSendData(topic2); - Map kafkaParams = new HashMap<>(); - kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); + Map kafkaParams = new HashMap<>(); + kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); + kafkaParams.put("key.deserializer", StringDeserializer.class); + kafkaParams.put("value.deserializer", StringDeserializer.class); OffsetRange[] offsetRanges = { OffsetRange.create(topic1, 0, 0, 1), OffsetRange.create(topic2, 0, 0, 1) }; - Map emptyLeaders = new HashMap<>(); - Map leaders = new HashMap<>(); + Map emptyLeaders = new HashMap<>(); + Map 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, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - offsetRanges - ).map( - new Function, String>() { - @Override - public String call(Tuple2 kv) { - return kv._2(); - } + String broker = hostAndPort[0]; + leaders.put(offsetRanges[0].topicPartition(), broker); + leaders.put(offsetRanges[1].topicPartition(), broker); + + Function, String> handler = + new Function, String>() { + @Override + public String call(ConsumerRecord r) { + return r.value(); } - ); + }; - JavaRDD rdd2 = KafkaUtils.createRDD( + JavaRDD rdd1 = KafkaRDD.create( sc, String.class, String.class, - StringDecoder.class, - StringDecoder.class, - String.class, kafkaParams, offsetRanges, - emptyLeaders, - new Function, String>() { - @Override - public String call(MessageAndMetadata msgAndMd) { - return msgAndMd.message(); - } - } - ); + leaders + ).map(handler); - JavaRDD rdd3 = KafkaUtils.createRDD( + JavaRDD rdd2 = KafkaRDD.create( sc, String.class, String.class, - StringDecoder.class, - StringDecoder.class, - String.class, kafkaParams, offsetRanges, - leaders, - new Function, String>() { - @Override - public String call(MessageAndMetadata msgAndMd) { - return msgAndMd.message(); - } - } - ); + emptyLeaders + ).map(handler); // 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) { diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index b2c81d1534ee6..4577b5e82b234 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -25,10 +25,12 @@ import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ import scala.concurrent.duration._ import scala.language.postfixOps +import scala.util.Random + +import org.apache.kafka.clients.consumer.{ ConsumerRecord, KafkaConsumer } +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.StringDeserializer -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.StringDecoder import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.concurrent.Eventually @@ -36,7 +38,6 @@ import org.apache.spark.{Logging, SparkConf, SparkContext, SparkFunSuite} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.scheduler.rate.RateEstimator import org.apache.spark.util.Utils @@ -82,26 +83,32 @@ class DirectKafkaStreamSuite } } + private def getKafkaParams(extra: (String, Object)*) = (Map[String, Object] ( + "bootstrap.servers" -> kafkaTestUtils.brokerAddress, + "key.deserializer" -> classOf[StringDeserializer], + "value.deserializer" -> classOf[StringDeserializer], + "group.id" -> s"test-consumer-${Random.nextInt}" + ) ++ extra).asJava + + private val preferredHosts = Map[TopicPartition, String]().asJava test("basic stream receiving with multiple topics and smallest starting offset") { - val topics = Set("basic1", "basic2", "basic3") + val topics = List("basic1", "basic2", "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( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics) + DirectKafkaInputDStream[String, String](ssc, kafkaParams, kafkaParams, preferredHosts) } + stream.subscribe(topics.asJava) + val allReceived = new ConcurrentLinkedQueue[(String, String)]() // hold a reference to the current offset ranges, so it can be used downstream @@ -110,7 +117,7 @@ class DirectKafkaStreamSuite stream.transform { rdd => // Get the offset ranges in the RDD offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - rdd + rdd.map(r => (r.key, r.value)) }.foreachRDD { rdd => for (o <- offsetRanges) { logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") @@ -131,7 +138,9 @@ class DirectKafkaStreamSuite assert(partSize === rangeSize, "offset ranges are wrong") } } - stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + stream.foreachRDD { rdd => + allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) + } ssc.start() eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { assert(allReceived.size === totalSent, @@ -142,17 +151,16 @@ class DirectKafkaStreamSuite } test("receiving from largest starting offset") { - val topic = "largest" - val topicPartition = TopicAndPartition(topic, 0) + val topic = "latest" + val topicPartition = new TopicPartition(topic, 0) val data = Map("a" -> 10) kafkaTestUtils.createTopic(topic) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "largest" - ) - val kc = new KafkaCluster(kafkaParams) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") + val kc = new KafkaConsumer(kafkaParams) + kc.assign(List(topicPartition).asJava) def getLatestOffset(): Long = { - kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + kc.seekToEnd(topicPartition) + kc.position(topicPartition) } // Send some initial messages before starting context @@ -161,21 +169,23 @@ class DirectKafkaStreamSuite assert(getLatestOffset() > 3) } val offsetBeforeStart = getLatestOffset() + kc.close() // 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, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) + DirectKafkaInputDStream[String, String]( + ssc, kafkaParams, kafkaParams, preferredHosts) } + stream.subscribe(List(topic).asJava) + stream.poll(0) assert( - stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] - .fromOffsets(topicPartition) >= offsetBeforeStart, + stream.position(topicPartition) >= offsetBeforeStart, "Start offset not from latest" ) val collectedData = new ConcurrentLinkedQueue[String]() - stream.map { _._2 }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + stream.map { _.value }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } ssc.start() val newData = Map("b" -> 10) kafkaTestUtils.sendMessages(topic, newData) @@ -188,16 +198,15 @@ class DirectKafkaStreamSuite test("creating stream by offset") { val topic = "offset" - val topicPartition = TopicAndPartition(topic, 0) + val topicPartition = new TopicPartition(topic, 0) val data = Map("a" -> 10) kafkaTestUtils.createTopic(topic) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "largest" - ) - val kc = new KafkaCluster(kafkaParams) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") + val kc = new KafkaConsumer(kafkaParams) + kc.assign(List(topicPartition).asJava) def getLatestOffset(): Long = { - kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset + kc.seekToEnd(topicPartition) + kc.position(topicPartition) } // Send some initial messages before starting context @@ -206,22 +215,24 @@ class DirectKafkaStreamSuite assert(getLatestOffset() >= 10) } val offsetBeforeStart = getLatestOffset() + kc.close() // 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, StringDecoder, StringDecoder, String]( - ssc, kafkaParams, Map(topicPartition -> 11L), - (m: MessageAndMetadata[String, String]) => m.message()) + DirectKafkaInputDStream[String, String]( + ssc, kafkaParams, kafkaParams, preferredHosts) } + stream.assign(List(topicPartition).asJava) + stream.seek(topicPartition, 11L) + assert( - stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] - .fromOffsets(topicPartition) >= offsetBeforeStart, + stream.position(topicPartition) >= offsetBeforeStart, "Start offset not from latest" ) val collectedData = new ConcurrentLinkedQueue[String]() - stream.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + stream.map(_.value).foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } ssc.start() val newData = Map("b" -> 10) kafkaTestUtils.sendMessages(topic, newData) @@ -237,10 +248,7 @@ class DirectKafkaStreamSuite kafkaTestUtils.createTopic(topic) testDir = Utils.createTempDir() - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { @@ -254,18 +262,20 @@ class DirectKafkaStreamSuite // Setup the streaming context ssc = new StreamingContext(sparkConf, Milliseconds(100)) val kafkaStream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) + val s = DirectKafkaInputDStream[String, String]( + ssc, kafkaParams, kafkaParams, preferredHosts) + s.subscribe(List(topic).asJava) + s } - val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } + val keyedStream = kafkaStream.map { r => "key" -> r.value.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() + kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String,String]], time: Time) => + val data = rdd.map { _.value }.collect() DirectKafkaStreamSuite.collectedData.addAll(Arrays.asList(data: _*)) } @@ -292,7 +302,8 @@ class DirectKafkaStreamSuite // Recover context from checkpoints ssc = new StreamingContext(testDir.getAbsolutePath) - val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] + val recoveredStream = + ssc.graph.getInputStreams().head.asInstanceOf[DStream[ConsumerRecord[String, String]]] // Verify offset ranges have been recovered val recoveredOffsetRanges = getOffsetRanges(recoveredStream) @@ -321,10 +332,7 @@ class DirectKafkaStreamSuite kafkaTestUtils.sendMessages(topic, data) val totalSent = data.values.sum - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") import DirectKafkaStreamSuite._ ssc = new StreamingContext(sparkConf, Milliseconds(200)) @@ -332,13 +340,15 @@ class DirectKafkaStreamSuite ssc.addStreamingListener(collector) val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) + DirectKafkaInputDStream[String, String]( + ssc, kafkaParams, kafkaParams, preferredHosts) } + stream.subscribe(List(topic).asJava) val allReceived = new ConcurrentLinkedQueue[(String, String)] - stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } + stream.map(r => (r.key, r.value)) + .foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } ssc.start() eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { assert(allReceived.size === totalSent, @@ -357,9 +367,9 @@ class DirectKafkaStreamSuite val topic = "maxMessagesPerPartition" val kafkaStream = getDirectKafkaStream(topic, None) - val input = Map(TopicAndPartition(topic, 0) -> 50L, TopicAndPartition(topic, 1) -> 50L) + val input = Map(new TopicPartition(topic, 0) -> 50L, new TopicPartition(topic, 1) -> 50L) assert(kafkaStream.maxMessagesPerPartition(input).get == - Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + Map(new TopicPartition(topic, 0) -> 10L, new TopicPartition(topic, 1) -> 10L)) } test("maxMessagesPerPartition with no lag") { @@ -367,7 +377,7 @@ class DirectKafkaStreamSuite val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 100)) val kafkaStream = getDirectKafkaStream(topic, rateController) - val input = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) + val input = Map(new TopicPartition(topic, 0) -> 0L, new TopicPartition(topic, 1) -> 0L) assert(kafkaStream.maxMessagesPerPartition(input).isEmpty) } @@ -376,19 +386,16 @@ class DirectKafkaStreamSuite val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 1000)) val kafkaStream = getDirectKafkaStream(topic, rateController) - val input = Map(TopicAndPartition(topic, 0) -> 1000L, TopicAndPartition(topic, 1) -> 1000L) + val input = Map(new TopicPartition(topic, 0) -> 1000L, new TopicPartition(topic, 1) -> 1000L) assert(kafkaStream.maxMessagesPerPartition(input).get == - Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) + Map(new TopicPartition(topic, 0) -> 10L, new TopicPartition(topic, 1) -> 10L)) } test("using rate controller") { val topic = "backpressure" - val topicPartitions = Set(TopicAndPartition(topic, 0), TopicAndPartition(topic, 1)) + val topicPartitions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) kafkaTestUtils.createTopic(topic, 2) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") val batchIntervalMilliseconds = 100 val estimator = new ConstantEstimator(100) @@ -406,16 +413,13 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) val kafkaStream = withClue("Error creating direct stream") { - val kc = new KafkaCluster(kafkaParams) - val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - val m = kc.getEarliestLeaderOffsets(topicPartitions) - .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) - - new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, kafkaParams, m, messageHandler) { + val s = new DirectKafkaInputDStream[String, String]( + ssc, kafkaParams, kafkaParams, preferredHosts) { override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) } + s.subscribe(List(topic).asJava) + s.map(r => (r.key, r.value)) } val collectedData = new ConcurrentLinkedQueue[Array[String]]() @@ -452,9 +456,9 @@ class DirectKafkaStreamSuite /** Get the generated offset ranges from the DirectKafkaStream */ private def getOffsetRanges[K, V]( - kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { + kafkaStream: DStream[ConsumerRecord[K, V]]): Seq[(Time, Array[OffsetRange])] = { kafkaStream.generatedRDDs.mapValues { rdd => - rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges + rdd.asInstanceOf[HasOffsetRanges].offsetRanges }.toSeq.sortBy { _._1 } } @@ -469,12 +473,18 @@ class DirectKafkaStreamSuite // Setup the streaming context ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) - val earliestOffsets = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) - val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, Map[String, String](), earliestOffsets, messageHandler) { + val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + + val s = new DirectKafkaInputDStream[String, String]( + ssc, kafkaParams, kafkaParams, preferredHosts) { override protected[streaming] val rateController = mockRateController } + val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + s.assign(tps.asJava) + tps.foreach(tp => s.seek(tp, 0)) + // manual start necessary because we arent consuming the stream, just checking its state + s.start() + s } } diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 5e539c1d790cc..a0eb8f8b01c64 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.streaming.kafka +import scala.collection.JavaConverters._ import scala.util.Random -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.StringDecoder +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.serialization.StringDeserializer import org.scalatest.BeforeAndAfterAll import org.apache.spark._ @@ -52,21 +52,28 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { } } + private def getKafkaParams() = Map[String, Object]( + "bootstrap.servers" -> kafkaTestUtils.brokerAddress, + "key.deserializer" -> classOf[StringDeserializer], + "value.deserializer" -> classOf[StringDeserializer], + "group.id" -> s"test-consumer-${Random.nextInt}" + ).asJava + + private val preferredHosts = Map[TopicPartition, String]().asJava + test("basic usage") { val topic = s"topicbasic-${Random.nextInt}" kafkaTestUtils.createTopic(topic) val messages = Array("the", "quick", "brown", "fox") kafkaTestUtils.sendMessages(topic, messages) - val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}") + val kafkaParams = getKafkaParams() val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) - val rdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, offsetRanges) + val rdd = KafkaRDD[String, String](sc, kafkaParams, offsetRanges, preferredHosts).map(_.value) - val received = rdd.map(_._2).collect.toSet + val received = rdd.collect.toSet assert(received === messages.toSet) // size-related method optimizations return sane results @@ -74,19 +81,20 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { 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(1).head === messages.head) assert(rdd.take(messages.size + 10).size === messages.size) - val emptyRdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0))) + val emptyRdd = KafkaRDD[String, String]( + sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0)), preferredHosts) assert(emptyRdd.isEmpty) // invalid offset ranges throw exceptions val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) intercept[SparkException] { - KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, badRanges) + val result = KafkaRDD[String, String](sc, kafkaParams, badRanges, preferredHosts) + .map(_.value) + .collect() } } @@ -96,80 +104,42 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) kafkaTestUtils.createTopic(topic) - val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}") - - val kc = new KafkaCluster(kafkaParams) + val kafkaParams = getKafkaParams() // this is the "lots of messages" case kafkaTestUtils.sendMessages(topic, sent) - val sentCount = sent.values.sum + var sentCount = sent.values.sum - // rdd defined from leaders after sending messages, should get the number sent - val rdd = getRdd(kc, Set(topic)) + val rdd = KafkaRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, 0, sentCount)), preferredHosts) - assert(rdd.isDefined) - - val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges + val ranges = rdd.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(kafkaParams("group.id"), rangesMap).fold( - err => throw new Exception(err.mkString("\n")), - _ => () - ) + assert(rdd.map(_.offset).collect.sorted === (0 until sentCount).toArray, + "didn't get all sent messages") // this is the "0 messages" case - val rdd2 = getRdd(kc, Set(topic)) + val rdd2 = KafkaRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, sentCount, sentCount)), preferredHosts) + // 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") + assert(rdd2.map(_.value).collect.size === 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)) + val rdd3 = KafkaRDD[String, String](sc, kafkaParams, + Array(OffsetRange(topic, 0, sentCount, sentCount + 1)), preferredHosts) + // 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") - + assert(rdd3.map(_.value).collect.head === sentOnlyOne.keys.head, + "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.getConsumerOffsets(groupId, topicPartitions).right.toOption.orElse( - kc.getEarliestLeaderOffsets(topicPartitions).right.toOption.map { offs => - offs.map(kv => kv._1 -> kv._2.offset) - } - ) - } - kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => - consumerOffsets(topicPartitions).flatMap { from => - kc.getLatestLeaderOffsets(topicPartitions).right.toOption.map { until => - val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) => - OffsetRange(tp.topic, tp.partition, fromOffset, until(tp).offset) - }.toArray - - val leaders = until.map { case (tp: TopicAndPartition, lo: KafkaCluster.LeaderOffset) => - tp -> Broker(lo.host, lo.port) - }.toMap - - KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder, String]( - sc, kc.kafkaParams, offsetRanges, leaders, - (mmd: MessageAndMetadata[String, String]) => s"${mmd.offset} ${mmd.message}") - } - } - } - } } From 43c27e929a40a331d2efc12e9e5cb2e18b44bcf3 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 11 Mar 2016 13:32:40 -0600 Subject: [PATCH 11/37] [SPARK-12177][Streaming][Kafka] WIP allow user configuration of consumer for dynamic topics, listener, etc --- .../kafka/DirectKafkaInputDStream.scala | 253 ++++-------------- .../spark/streaming/kafka/KafkaRDD.scala | 4 + .../kafka/JavaDirectKafkaStreamSuite.java | 38 ++- .../kafka/DirectKafkaStreamSuite.scala | 120 +++++---- 4 files changed, 150 insertions(+), 265 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 1abff4e6a3843..8cfd32e9e1636 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import java.{ util => ju } import org.apache.kafka.clients.consumer.{ - ConsumerConfig, ConsumerRebalanceListener, ConsumerRecord, KafkaConsumer + ConsumerConfig, ConsumerRebalanceListener, ConsumerRecord, Consumer } import org.apache.kafka.common.{ PartitionInfo, TopicPartition } @@ -53,156 +53,44 @@ import scala.collection.JavaConverters._ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( _ssc: StreamingContext, - val driverKafkaParams: ju.Map[String, Object], - val executorKafkaParams: ju.Map[String, Object], - preferredHosts: ju.Map[TopicPartition, String] + preferredHosts: ju.Map[TopicPartition, String], + executorKafkaParams: ju.Map[String, Object], + driverConsumer: () => Consumer[K, V] ) extends InputDStream[ConsumerRecord[K,V]](_ssc) with Logging { - import DirectKafkaInputDStream.{ - PartitionAssignment, Assigned, Subscribed, PatternSubscribed, Unassigned - } - - assert(false == - driverKafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], - ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + - " must be set to false for driver kafka params, else offsets may commit before processing") - - @transient private var kc: KafkaConsumer[K, V] = null - private var partitionAssignment: PartitionAssignment = Unassigned - protected def consumer(): KafkaConsumer[K, V] = this.synchronized { + @transient private var kc: Consumer[K, V] = null + def consumer(): Consumer[K, V] = this.synchronized { if (null == kc) { - kc = new KafkaConsumer(driverKafkaParams) - assignPartitions(partitionAssignment) + kc = driverConsumer() } kc } consumer() - // XXX TODO listeners arent much use if they dont have a reference to the consumer - private def listenerFor(className: String): ConsumerRebalanceListener = - Class.forName(className) - .newInstance() - .asInstanceOf[ConsumerRebalanceListener] - - private def assignPartitions(pa: PartitionAssignment): Unit = this.synchronized { - import DirectKafkaInputDStream.defaultListener - - val reset = driverKafkaParams.get(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).asInstanceOf[String] - .toLowerCase - val resetMsg = "Dynamic topic subscriptions may not work well unless " + - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + " is set or a rebalance listener is configured" - - // using kc directly because consumer() calls this method - pa match { - case Assigned(partitions) => - kc.assign(partitions) - case Subscribed(topics, className) => - if (!(reset == "earliest" || reset == "latest" || className != defaultListener)) { - log.warn(resetMsg) - } - kc.subscribe(topics, listenerFor(className)) - case PatternSubscribed(pattern, className) => - if (!(reset == "earliest" || reset == "latest" || className != defaultListener)) { - log.warn(resetMsg) + protected def getBrokers = { + val c = consumer + val result = new ju.HashMap[TopicPartition, String]() + val hosts = new ju.HashMap[TopicPartition, String]() + val assignments = c.assignment().iterator() + while (assignments.hasNext()) { + val tp: TopicPartition = assignments.next() + if (null == hosts.get(tp)) { + val infos = c.partitionsFor(tp.topic).iterator() + while (infos.hasNext()) { + val i = infos.next() + hosts.put(new TopicPartition(i.topic(), i.partition()), i.leader.host()) } - kc.subscribe(pattern, listenerFor(className)) - case Unassigned => + } + result.put(tp, hosts.get(tp)) } - - this.partitionAssignment = pa - } - - /** Manually assign a list of partitions */ - def assign(partitions: ju.List[TopicPartition]): Unit = { - assignPartitions(Assigned(partitions)) - } - - /** Subscribe to the given list of topics to get dynamically assigned partitions */ - def subscribe(topics: ju.List[String]): Unit = { - assignPartitions(Subscribed(topics)) - } - - /** Subscribe to the given list of topics to get dynamically assigned partitions */ - def subscribe( - topics: ju.List[String], - consumerRebalanceListenerClassName: String): Unit = { - assignPartitions(Subscribed(topics, consumerRebalanceListenerClassName)) - } - - /** Subscribe to all topics matching specified pattern to get dynamically assigned partitions. - * The pattern matching will be done periodically against topics existing at the time of check. - */ - def subscribe(pattern: ju.regex.Pattern): Unit = { - assignPartitions(PatternSubscribed(pattern)) - } - - /** Subscribe to all topics matching specified pattern to get dynamically assigned partitions. - * The pattern matching will be done periodically against topics existing at the time of check. - */ - def subscribe( - pattern: ju.regex.Pattern, - consumerRebalanceListenerClassName: String): Unit = { - assignPartitions(PatternSubscribed(pattern, consumerRebalanceListenerClassName)) - } - - /** Get the set of partitions currently assigned to the underlying consumer */ - def assignment(): ju.Set[TopicPartition] = this.synchronized { - consumer.assignment() - } - - /** Get metadata about the partitions for a given topic. */ - def partitionsFor(topic: String): ju.List[PartitionInfo] = this.synchronized { - consumer.partitionsFor(topic) - } - - /** Necessary to fetch metadata and update subscriptions, - * driver shouldn't be reading messages, so don't call this with non-zero timeout */ - def poll(timeout: Long): Unit = this.synchronized { - consumer.poll(timeout) + result } - /** Get the offset of the next record that will be fetched - * (if a record with that offset exists). - */ - def position(partition: TopicPartition): Long = this.synchronized { - consumer.position(partition) - } - - def seek(partition: TopicPartition, offset: Long): Unit = this.synchronized { - consumer.seek(partition, offset) - } - - def seekToBeginning(partitions: TopicPartition*): Unit = this.synchronized { - consumer.seekToBeginning(partitions: _*) - } - - def seekToEnd(partitions: TopicPartition*): Unit = this.synchronized { - consumer.seekToEnd(partitions: _*) - } - - // TODO is there a better way to distinguish between - // - want to use leader brokers (null map) - // - don't care, use consistent executor (empty map) - // - want to use specific hosts (non-null, non-empty map) - private def getPreferredHosts: ju.Map[TopicPartition, String] = { - if (null != preferredHosts) { - preferredHosts + protected def getPreferredHosts: ju.Map[TopicPartition, String] = { + if (preferredHosts == DirectKafkaInputDStream.preferBrokers) { + getBrokers } else { - val result = new ju.HashMap[TopicPartition, String]() - val hosts = new ju.HashMap[TopicPartition, String]() - val assignments = assignment().iterator() - while (assignments.hasNext()) { - val a = assignments.next() - if (null == hosts.get(a)) { - val infos = partitionsFor(a.topic).iterator() - while (infos.hasNext()) { - val i = infos.next() - hosts.put(new TopicPartition(i.topic(), i.partition()), i.leader.host()) - } - } - result.put(a, hosts.get(a)) - } - result + preferredHosts } } @@ -260,15 +148,15 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( protected var currentOffsets = Map[TopicPartition, Long]() - protected def latestOffsets(): Map[TopicPartition, Long] = this.synchronized { + protected def latestOffsets(): Map[TopicPartition, Long] = { val c = consumer c.poll(0) val parts = c.assignment().asScala - if (!partitionAssignment.isInstanceOf[Assigned]) { - // make sure new partitions are reflected in currentOffsets - val newPartitions = parts.diff(currentOffsets.keySet) - currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap - } + + // make sure new partitions are reflected in currentOffsets + val newPartitions = parts.diff(currentOffsets.keySet) + currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap + c.seekToEnd() parts.map(tp => tp -> c.position(tp)).toMap } @@ -314,8 +202,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( Some(rdd) } - override def start(): Unit = this.synchronized { - assert(partitionAssignment != Unassigned, "Must call subscribe or assign before starting") + override def start(): Unit = { val c = consumer c.poll(0) if (currentOffsets.isEmpty) { @@ -325,9 +212,9 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } } - override def stop(): Unit = { - this.synchronized { - consumer.close() + override def stop(): Unit = this.synchronized { + if (kc != null) { + kc.close() } } @@ -348,8 +235,6 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( override def cleanup(time: Time) { } override def restore() { - poll(0) - 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]( @@ -369,72 +254,36 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( object DirectKafkaInputDStream extends Logging { import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } + import org.apache.spark.api.java.function.{ Function0 => JFunction0 } - protected val defaultListener = - "org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener" - /** There are several different ways of specifying partition assignment, - * and they need to be able to survive checkpointing - */ - protected sealed trait PartitionAssignment extends Serializable - /** manual assignment via consumer.assign() */ - protected case class Assigned(partitions: ju.List[TopicPartition]) extends PartitionAssignment - /** dynamic subscription to list of topics via consumer.subscribe */ - protected case class Subscribed( - topics: ju.List[String], - consumerRebalanceListenerClassName: String = defaultListener) extends PartitionAssignment - /** dynamic subscription to topics matching pattern via consumer.subscribe */ - protected case class PatternSubscribed( - pattern: ju.regex.Pattern, - consumerRebalanceListenerClassName: String = defaultListener) extends PartitionAssignment - /** Not yet assigned */ - protected case object Unassigned extends PartitionAssignment - - private def fixKafkaParams( - dkp: ju.HashMap[String, Object], - ekp: ju.HashMap[String, Object] - ): Unit = { - // wouldn't this be nice to have... 0.10 has it, but with a minimum of 1 - // log.warn(s"overriding ${ConsumerConfig.MAX_POLL_RECORDS_CONFIG} to 0 for driver") - // dkp.put(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, 0: Integer) - - log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for driver") - dkp.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) - - // this probably doesnt matter since executors are manually assigned partitions, but just in case - if (null != ekp.get(ConsumerConfig.GROUP_ID_CONFIG) && - ekp.get(ConsumerConfig.GROUP_ID_CONFIG) == - dkp.get(ConsumerConfig.GROUP_ID_CONFIG) - ) { - val id = ekp.get(ConsumerConfig.GROUP_ID_CONFIG) + "-executor" - log.warn(s"overriding ${ConsumerConfig.GROUP_ID_CONFIG} to ${id} for executor") - ekp.put(ConsumerConfig.GROUP_ID_CONFIG, id) - } - - } + /** Prefer to run on kafka brokers, if they are on same hosts as executors */ + val preferBrokers: ju.Map[TopicPartition, String] = null + /** Prefer a consistent executor per TopicPartition, evenly from all executors */ + val preferConsistent: ju.Map[TopicPartition, String] = ju.Collections.emptyMap() + /** Scala constructor */ def apply[K: ClassTag, V: ClassTag]( ssc: StreamingContext, - driverKafkaParams: ju.Map[String, Object], + preferredHosts: ju.Map[TopicPartition, String], executorKafkaParams: ju.Map[String, Object], - preferredHosts: ju.Map[TopicPartition, String] + driverConsumer: () => Consumer[K,V] ): DirectKafkaInputDStream[K, V] = { - val dkp = new ju.HashMap[String, Object](driverKafkaParams) - val ekp = new ju.HashMap[String, Object](executorKafkaParams) val ph = new ju.HashMap[TopicPartition, String](preferredHosts) - - fixKafkaParams(dkp, ekp) + val ekp = new ju.HashMap[String, Object](executorKafkaParams) KafkaRDD.fixKafkaParams(ekp) + val cleaned = ssc.sparkContext.clean(driverConsumer) - new DirectKafkaInputDStream[K, V](ssc, dkp, ekp, ph) + new DirectKafkaInputDStream[K, V](ssc, ph, ekp, cleaned) } + /** Java constructor */ def create[K, V]( jssc: JavaStreamingContext, keyClass: Class[K], valueClass: Class[V], - driverKafkaParams: ju.Map[String, Object], + preferredHosts: ju.Map[TopicPartition, String], executorKafkaParams: ju.Map[String, Object], - preferredHosts: ju.Map[TopicPartition, String] + driverConsumer: JFunction0[Consumer[K, V]] ): JavaInputDStream[ConsumerRecord[K, V]] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) @@ -442,7 +291,7 @@ object DirectKafkaInputDStream extends Logging { new JavaInputDStream( DirectKafkaInputDStream[K, V]( - jssc.ssc, driverKafkaParams, executorKafkaParams, preferredHosts)) + jssc.ssc, preferredHosts, executorKafkaParams, driverConsumer.call _)) } } diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 23fe50d689f07..5a110114d651a 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -209,6 +209,10 @@ object KafkaRDD extends Logging { offsetRanges: Array[OffsetRange], preferredHosts: ju.Map[TopicPartition, String] ): KafkaRDD[K, V] = { + assert(preferredHosts != DirectKafkaInputDStream.preferBrokers, + "If you want to prefer brokers, you must provide a mapping for preferredHosts. " + + "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you. ") + val kp = new ju.HashMap[String, Object](kafkaParams) fixKafkaParams(kp) val osr = offsetRanges.clone() diff --git a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index 22c2060e48eeb..d1e8d794ba58b 100644 --- a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -24,6 +24,8 @@ import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.KafkaConsumer; import org.junit.After; import org.junit.Assert; @@ -34,6 +36,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function0; import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.api.java.JavaDStream; @@ -82,26 +85,31 @@ public void testKafkaStream() throws InterruptedException { Random random = new Random(); - Map kafkaParams = new HashMap<>(); + final Map kafkaParams = new HashMap<>(); kafkaParams.put("bootstrap.servers", kafkaTestUtils.brokerAddress()); kafkaParams.put("key.deserializer", StringDeserializer.class); kafkaParams.put("value.deserializer", StringDeserializer.class); kafkaParams.put("auto.offset.reset", "earliest"); kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); - Map preferredHosts = new HashMap<>(); + Map preferredHosts = DirectKafkaInputDStream.preferConsistent(); JavaInputDStream istream1 = DirectKafkaInputDStream.create( ssc, String.class, String.class, + preferredHosts, kafkaParams, - kafkaParams, - preferredHosts + new Function0>() { + @Override + public Consumer call() { + KafkaConsumer consumer = new KafkaConsumer(kafkaParams); + consumer.subscribe(Arrays.asList(topic1)); + return consumer; + } + } ); - ((DirectKafkaInputDStream) istream1.inputDStream()).subscribe(Arrays.asList(topic1)); - JavaDStream stream1 = istream1.transform( // Make sure you can get offset ranges from the rdd new Function>, @@ -125,19 +133,25 @@ public String call(ConsumerRecord r) { } ); - kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); + final Map kafkaParams2 = new HashMap<>(kafkaParams); + kafkaParams2.put("group.id", "java-test-consumer-" + random.nextInt()); JavaInputDStream istream2 = DirectKafkaInputDStream.create( ssc, String.class, String.class, - kafkaParams, - kafkaParams, - preferredHosts + preferredHosts, + kafkaParams2, + new Function0>() { + @Override + public Consumer call() { + KafkaConsumer consumer = new KafkaConsumer(kafkaParams2); + consumer.subscribe(Arrays.asList(topic2)); + return consumer; + } + } ); - ((DirectKafkaInputDStream) istream2.inputDStream()).subscribe(Arrays.asList(topic2)); - JavaDStream stream2 = istream2.transform( // Make sure you can get offset ranges from the rdd new Function>, diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 4577b5e82b234..6d103ebc632f3 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.kafka import java.io.File -import java.util.Arrays +import java.util.{ Arrays, HashMap => JHashMap } import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.ConcurrentLinkedQueue @@ -83,14 +83,17 @@ class DirectKafkaStreamSuite } } - private def getKafkaParams(extra: (String, Object)*) = (Map[String, Object] ( - "bootstrap.servers" -> kafkaTestUtils.brokerAddress, - "key.deserializer" -> classOf[StringDeserializer], - "value.deserializer" -> classOf[StringDeserializer], - "group.id" -> s"test-consumer-${Random.nextInt}" - ) ++ extra).asJava + def getKafkaParams(extra: (String, Object)*) = { + val kp = new JHashMap[String, Object]() + kp.put("bootstrap.servers", kafkaTestUtils.brokerAddress) + kp.put("key.deserializer", classOf[StringDeserializer]) + kp.put("value.deserializer", classOf[StringDeserializer]) + kp.put("group.id", s"test-consumer-${Random.nextInt}") + extra.foreach(e => kp.put(e._1, e._2)) + kp + } - private val preferredHosts = Map[TopicPartition, String]().asJava + val preferredHosts = DirectKafkaInputDStream.preferConsistent test("basic stream receiving with multiple topics and smallest starting offset") { val topics = List("basic1", "basic2", "basic3") @@ -104,21 +107,23 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - DirectKafkaInputDStream[String, String](ssc, kafkaParams, kafkaParams, preferredHosts) + DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topics: _*)) + consumer + }) } - - stream.subscribe(topics.asJava) - val allReceived = new ConcurrentLinkedQueue[(String, String)]() // hold a reference to the current offset ranges, so it can be used downstream var offsetRanges = Array[OffsetRange]() - - stream.transform { rdd => + val tf = stream.transform { rdd => // Get the offset ranges in the RDD offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges rdd.map(r => (r.key, r.value)) - }.foreachRDD { rdd => + } + + tf.foreachRDD { rdd => for (o <- offsetRanges) { logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") } @@ -138,6 +143,7 @@ class DirectKafkaStreamSuite assert(partSize === rangeSize, "offset ranges are wrong") } } + stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) } @@ -157,7 +163,7 @@ class DirectKafkaStreamSuite kafkaTestUtils.createTopic(topic) val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") val kc = new KafkaConsumer(kafkaParams) - kc.assign(List(topicPartition).asJava) + kc.assign(Arrays.asList(topicPartition)) def getLatestOffset(): Long = { kc.seekToEnd(topicPartition) kc.position(topicPartition) @@ -174,15 +180,18 @@ class DirectKafkaStreamSuite // Setup context and kafka stream with largest offset ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - DirectKafkaInputDStream[String, String]( - ssc, kafkaParams, kafkaParams, preferredHosts) + val s = DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer + }) + s.consumer.poll(0) + assert( + s.consumer.position(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + s } - stream.subscribe(List(topic).asJava) - stream.poll(0) - assert( - stream.position(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) val collectedData = new ConcurrentLinkedQueue[String]() stream.map { _.value }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } @@ -203,7 +212,7 @@ class DirectKafkaStreamSuite kafkaTestUtils.createTopic(topic) val kafkaParams = getKafkaParams("auto.offset.reset" -> "latest") val kc = new KafkaConsumer(kafkaParams) - kc.assign(List(topicPartition).asJava) + kc.assign(Arrays.asList(topicPartition)) def getLatestOffset(): Long = { kc.seekToEnd(topicPartition) kc.position(topicPartition) @@ -220,16 +229,19 @@ class DirectKafkaStreamSuite // Setup context and kafka stream with largest offset ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - DirectKafkaInputDStream[String, String]( - ssc, kafkaParams, kafkaParams, preferredHosts) + val s = DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.assign(Arrays.asList(topicPartition)) + consumer.seek(topicPartition, 11L) + consumer + }) + s.consumer.poll(0) + assert( + s.consumer.position(topicPartition) >= offsetBeforeStart, + "Start offset not from latest" + ) + s } - stream.assign(List(topicPartition).asJava) - stream.seek(topicPartition, 11L) - - assert( - stream.position(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) val collectedData = new ConcurrentLinkedQueue[String]() stream.map(_.value).foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } @@ -262,10 +274,11 @@ class DirectKafkaStreamSuite // Setup the streaming context ssc = new StreamingContext(sparkConf, Milliseconds(100)) val kafkaStream = withClue("Error creating direct stream") { - val s = DirectKafkaInputDStream[String, String]( - ssc, kafkaParams, kafkaParams, preferredHosts) - s.subscribe(List(topic).asJava) - s + DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer + }) } val keyedStream = kafkaStream.map { r => "key" -> r.value.toInt } val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => @@ -340,10 +353,12 @@ class DirectKafkaStreamSuite ssc.addStreamingListener(collector) val stream = withClue("Error creating direct stream") { - DirectKafkaInputDStream[String, String]( - ssc, kafkaParams, kafkaParams, preferredHosts) + DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer + }) } - stream.subscribe(List(topic).asJava) val allReceived = new ConcurrentLinkedQueue[(String, String)] @@ -413,13 +428,14 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) val kafkaStream = withClue("Error creating direct stream") { - val s = new DirectKafkaInputDStream[String, String]( - ssc, kafkaParams, kafkaParams, preferredHosts) { + new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer + }){ override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) - } - s.subscribe(List(topic).asJava) - s.map(r => (r.key, r.value)) + }.map(r => (r.key, r.value)) } val collectedData = new ConcurrentLinkedQueue[Array[String]]() @@ -475,13 +491,15 @@ class DirectKafkaStreamSuite val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") - val s = new DirectKafkaInputDStream[String, String]( - ssc, kafkaParams, kafkaParams, preferredHosts) { + val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + consumer.assign(Arrays.asList(tps: _*)) + tps.foreach(tp => consumer.seek(tp, 0)) + consumer + }){ override protected[streaming] val rateController = mockRateController } - val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) - s.assign(tps.asJava) - tps.foreach(tp => s.seek(tp, 0)) // manual start necessary because we arent consuming the stream, just checking its state s.start() s From c04bc68726a5a9fb033e8a9cf57debf306d87a62 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 15 Mar 2016 11:03:02 -0500 Subject: [PATCH 12/37] [SPARK-12177][Streaming][Kafka] WIP allow manual commit of offsets to kafka --- .../kafka/DirectKafkaInputDStream.scala | 43 +++++++++++- .../kafka/DirectKafkaStreamSuite.scala | 65 ++++++++++++++++++- 2 files changed, 103 insertions(+), 5 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 8cfd32e9e1636..55057873af7fa 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -22,10 +22,10 @@ import scala.collection.mutable import scala.reflect.ClassTag import java.{ util => ju } +import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.atomic.AtomicReference -import org.apache.kafka.clients.consumer.{ - ConsumerConfig, ConsumerRebalanceListener, ConsumerRecord, Consumer -} +import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.{ PartitionInfo, TopicPartition } import org.apache.spark.{Logging, SparkException} @@ -199,6 +199,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) currentOffsets = untilOffsets + commitAll() Some(rdd) } @@ -218,6 +219,42 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } } + protected val commitQueue = new ConcurrentLinkedQueue[OffsetRange] + protected val commitCallback = new AtomicReference[OffsetCommitCallback] + + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange]): Unit = { + commitAsync(offsetRanges, null) + } + + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + * @param callback Only the most recently provided callback will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit = { + commitCallback.set(callback) + commitQueue.addAll(ju.Arrays.asList(offsetRanges: _*)) + } + + protected def commitAll(): Unit = { + val m = new ju.HashMap[TopicPartition, OffsetAndMetadata]() + val it = commitQueue.iterator() + while (it.hasNext) { + val osr = it.next + val tp = osr.topicPartition + val x = m.get(tp) + val offset = if (null == x) { osr.untilOffset } else { Math.max(x.offset, osr.untilOffset) } + m.put(tp, new OffsetAndMetadata(offset)) + } + if (!m.isEmpty) { + consumer.commitAsync(m, commitCallback.get) + } + } + private[streaming] class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 6d103ebc632f3..3b014d0282920 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming.kafka import java.io.File -import java.util.{ Arrays, HashMap => JHashMap } +import java.util.{ Arrays, HashMap => JHashMap, Map => JMap } import java.util.concurrent.atomic.AtomicLong import java.util.concurrent.ConcurrentLinkedQueue @@ -27,7 +27,7 @@ import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random -import org.apache.kafka.clients.consumer.{ ConsumerRecord, KafkaConsumer } +import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.StringDeserializer @@ -338,6 +338,67 @@ class DirectKafkaStreamSuite ssc.stop() } + // Test to verify the offsets can be recovered from Kafka + test("offset recovery from kafka") { + val topic = "recoveryfromkafka" + kafkaTestUtils.createTopic(topic) + + val kafkaParams = getKafkaParams( + "auto.offset.reset" -> "earliest", + ("enable.auto.commit", false: java.lang.Boolean) + ) + + val collectedData = new ConcurrentLinkedQueue[String]() + val committed = new JHashMap[TopicPartition, OffsetAndMetadata]() + + // 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 { collectedData.contains }) + } + } + + // Setup the streaming context + ssc = new StreamingContext(sparkConf, Milliseconds(100)) + withClue("Error creating direct stream") { + val kafkaStream = DirectKafkaInputDStream[String, String]( + ssc, preferredHosts, kafkaParams, () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer + }) + kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => + val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges + val data = rdd.map(_.value).collect() + collectedData.addAll(Arrays.asList(data: _*)) + kafkaStream.commitAsync(offsets, new OffsetCommitCallback() { + def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { + committed.putAll(m) + } + }) + } + } + ssc.start() + // Send some data and wait for them to be received + for (i <- (1 to 10).grouped(4)) { + sendDataAndWaitForReceive(i) + } + ssc.stop() + assert(! committed.isEmpty) + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer.poll(0) + committed.asScala.foreach { + case (k, v) => + // commits are async, not exactly once + assert(v.offset > 0) + assert(consumer.position(k) >= v.offset) + } + } + + test("Direct Kafka stream report input information") { val topic = "report-test" val data = Map("a" -> 7, "b" -> 9) From 546246e6c40e7b0960af0d9e7c02623421d2009f Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 15 Mar 2016 11:36:36 -0500 Subject: [PATCH 13/37] [SPARK-12177][Streaming][Kafka] WIP pause topicpartitions to prevent consuming messages on driver --- .../spark/streaming/kafka/DirectKafkaInputDStream.scala | 4 ++++ .../spark/streaming/kafka/DirectKafkaStreamSuite.scala | 6 +++++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 55057873af7fa..93a2df9f5390b 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -156,6 +156,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( // make sure new partitions are reflected in currentOffsets val newPartitions = parts.diff(currentOffsets.keySet) currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap + newPartitions.foreach(tp => c.pause(tp)) c.seekToEnd() parts.map(tp => tp -> c.position(tp)).toMap @@ -211,6 +212,9 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( tp -> c.position(tp) }.toMap } + + // don't actually want to consume any messages, so pause all partitions + currentOffsets.keySet.foreach(tp => c.pause(tp)) } override def stop(): Unit = this.synchronized { diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 3b014d0282920..bac268a6a90c9 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -375,7 +375,11 @@ class DirectKafkaStreamSuite collectedData.addAll(Arrays.asList(data: _*)) kafkaStream.commitAsync(offsets, new OffsetCommitCallback() { def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { - committed.putAll(m) + if (null != e) { + System.err.println(e) + } else { + committed.putAll(m) + } } }) } From 8f2da8c60a10848724b75e64363fd8fb5d37bf4d Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Mon, 21 Mar 2016 11:03:51 -0500 Subject: [PATCH 14/37] [SPARK-12177][Streaming][Kafka] scalastyle cleanup --- .../streaming/kafka/DirectKafkaInputDStream.scala | 15 +++++++-------- .../apache/spark/streaming/kafka/KafkaRDD.scala | 7 ++++--- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 93a2df9f5390b..a40172d2d2bf5 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -17,14 +17,15 @@ package org.apache.spark.streaming.kafka -import scala.annotation.tailrec -import scala.collection.mutable -import scala.reflect.ClassTag - import java.{ util => ju } import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicReference +import scala.annotation.tailrec +import scala.collection.JavaConverters._ +import scala.collection.mutable +import scala.reflect.ClassTag + import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.{ PartitionInfo, TopicPartition } @@ -34,8 +35,6 @@ import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} import org.apache.spark.streaming.scheduler.rate.RateEstimator -import scala.collection.JavaConverters._ - /** * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where * each given Kafka topic/partition corresponds to an RDD partition. @@ -56,7 +55,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( preferredHosts: ju.Map[TopicPartition, String], executorKafkaParams: ju.Map[String, Object], driverConsumer: () => Consumer[K, V] - ) extends InputDStream[ConsumerRecord[K,V]](_ssc) with Logging { + ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging { @transient private var kc: Consumer[K, V] = null def consumer(): Consumer[K, V] = this.synchronized { @@ -307,7 +306,7 @@ object DirectKafkaInputDStream extends Logging { ssc: StreamingContext, preferredHosts: ju.Map[TopicPartition, String], executorKafkaParams: ju.Map[String, Object], - driverConsumer: () => Consumer[K,V] + driverConsumer: () => Consumer[K, V] ): DirectKafkaInputDStream[K, V] = { val ph = new ju.HashMap[TopicPartition, String](preferredHosts) val ekp = new ju.HashMap[String, Object](executorKafkaParams) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 5a110114d651a..27b19266df97d 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -17,11 +17,11 @@ package org.apache.spark.streaming.kafka +import java.{ util => ju } + import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} -import java.{ util => ju } - import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } import org.apache.kafka.common.TopicPartition @@ -34,7 +34,8 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation * 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 + * @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 From ba41956a0b226a4841cf9d03344a6b8049fcad9f Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Mon, 21 Mar 2016 11:57:33 -0500 Subject: [PATCH 15/37] [SPARK-12177][Streaming][Kafka] scalastyle test cleanup --- .../streaming/kafka/DirectKafkaStreamSuite.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index ed981d54bace4..be32e048847cb 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -30,7 +30,6 @@ import scala.util.Random import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.StringDeserializer - import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} import org.scalatest.concurrent.Eventually @@ -84,7 +83,7 @@ class DirectKafkaStreamSuite } } - def getKafkaParams(extra: (String, Object)*) = { + def getKafkaParams(extra: (String, Object)*): JHashMap[String, Object] = { val kp = new JHashMap[String, Object]() kp.put("bootstrap.servers", kafkaTestUtils.brokerAddress) kp.put("key.deserializer", classOf[StringDeserializer]) @@ -195,7 +194,9 @@ class DirectKafkaStreamSuite } val collectedData = new ConcurrentLinkedQueue[String]() - stream.map { _.value }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } + stream.map { _.value }.foreachRDD { rdd => + collectedData.addAll(Arrays.asList(rdd.collect(): _*)) + } ssc.start() val newData = Map("b" -> 10) kafkaTestUtils.sendMessages(topic, newData) @@ -288,7 +289,7 @@ class DirectKafkaStreamSuite ssc.checkpoint(testDir.getAbsolutePath) // This is to collect the raw data received from Kafka - kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String,String]], time: Time) => + kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => val data = rdd.map { _.value }.collect() DirectKafkaStreamSuite.collectedData.addAll(Arrays.asList(data: _*)) } @@ -377,7 +378,7 @@ class DirectKafkaStreamSuite kafkaStream.commitAsync(offsets, new OffsetCommitCallback() { def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { if (null != e) { - System.err.println(e) + logError("commit failed", e) } else { committed.putAll(m) } @@ -498,7 +499,7 @@ class DirectKafkaStreamSuite val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topic)) consumer - }){ + }) { override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) }.map(r => (r.key, r.value)) @@ -563,7 +564,7 @@ class DirectKafkaStreamSuite consumer.assign(Arrays.asList(tps: _*)) tps.foreach(tp => consumer.seek(tp, 0)) consumer - }){ + }) { override protected[streaming] val rateController = mockRateController } // manual start necessary because we arent consuming the stream, just checking its state From e559183758db837aee4868283e9bc538e9af7c58 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Thu, 7 Apr 2016 17:10:43 -0500 Subject: [PATCH 16/37] [SPARK-12177][Streaming][Kafka] workaround performance issues, error on attempts to serialize ConsumerRecord --- .../kafka/DirectKafkaInputDStream.scala | 10 +++++-- .../spark/streaming/kafka/KafkaRDD.scala | 14 ++++++++++ .../kafka/DirectKafkaStreamSuite.scala | 26 ++++++++++++------- 3 files changed, 39 insertions(+), 11 deletions(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 4f20e96652c6d..6da4ba029c206 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -31,6 +31,7 @@ import org.apache.kafka.common.{ PartitionInfo, TopicPartition } import org.apache.spark.SparkException import org.apache.spark.internal.Logging +import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} @@ -67,6 +68,12 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } consumer() + override def persist(newLevel: StorageLevel): DStream[ConsumerRecord[K, V]] = { + log.error("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + protected def getBrokers = { val c = consumer val result = new ju.HashMap[TopicPartition, String]() @@ -180,8 +187,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( val fo = currentOffsets(tp) OffsetRange(tp.topic, tp.partition, fo, uo) } - - val rdd = KafkaRDD[K, V]( + val rdd = new KafkaRDD[K, V]( context.sparkContext, executorKafkaParams, offsetRanges.toArray, getPreferredHosts) // Report the record number and metadata of this batch interval to InputInfoTracker. diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index eb4b948edcc0c..9d2deb125a83e 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.ExecutorCacheTaskLocation +import org.apache.spark.storage.StorageLevel /** * A batch-oriented interface for consuming from Kafka. @@ -70,6 +71,12 @@ class KafkaRDD[ private val cacheLoadFactor = conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat + override def persist(newLevel: StorageLevel): this.type = { + log.error("Kafka ConsumerRecord is not serializable. " + + "Use .map to extract fields before calling .persist or .window") + super.persist(newLevel) + } + override def getPartitions: Array[Partition] = { offsetRanges.zipWithIndex.map { case (o, i) => new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset) @@ -203,6 +210,13 @@ object KafkaRDD extends Logging { log.warn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // possible workaround for KAFKA-3135 + val rbb = kafkaParams.get(ConsumerConfig.RECEIVE_BUFFER_CONFIG) + if (null == rbb || rbb.asInstanceOf[java.lang.Integer] < 65536) { + log.warn(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 for KAFKA-3135") + kafkaParams.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + } } def apply[K: ClassTag, V: ClassTag]( diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index be32e048847cb..ffc647b341a5d 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -478,6 +478,8 @@ class DirectKafkaStreamSuite val topicPartitions = Set(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) kafkaTestUtils.createTopic(topic, 2) val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") + val executorKafkaParams = new JHashMap[String, Object](kafkaParams) + KafkaRDD.fixKafkaParams(executorKafkaParams) val batchIntervalMilliseconds = 100 val estimator = new ConstantEstimator(100) @@ -495,7 +497,7 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) val kafkaStream = withClue("Error creating direct stream") { - new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + new DirectKafkaInputDStream[String, String](ssc, preferredHosts, executorKafkaParams, () => { val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topic)) consumer @@ -557,14 +559,20 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") - - val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) - consumer.assign(Arrays.asList(tps: _*)) - tps.foreach(tp => consumer.seek(tp, 0)) - consumer - }) { + val executorKafkaParams = new JHashMap[String, Object](kafkaParams) + KafkaRDD.fixKafkaParams(executorKafkaParams) + + val s = new DirectKafkaInputDStream[String, String]( + ssc, + preferredHosts, + executorKafkaParams, + () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + consumer.assign(Arrays.asList(tps: _*)) + tps.foreach(tp => consumer.seek(tp, 0)) + consumer + }) { override protected[streaming] val rateController = mockRateController } // manual start necessary because we arent consuming the stream, just checking its state From 47d3a9f96a63234b7b781f7881ae91bf034afc60 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 8 Apr 2016 10:57:55 -0500 Subject: [PATCH 17/37] [SPARK-12177][Streaming][Kafka] keep executor consumer group distinct from driver --- .../scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 9d2deb125a83e..144a6d4ea7173 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -211,10 +211,15 @@ object KafkaRDD extends Logging { log.warn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + // driver and executor should be in different consumer groups + val groupId = "spark-executor-" + kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + log.warn(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") + kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + // possible workaround for KAFKA-3135 val rbb = kafkaParams.get(ConsumerConfig.RECEIVE_BUFFER_CONFIG) if (null == rbb || rbb.asInstanceOf[java.lang.Integer] < 65536) { - log.warn(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 for KAFKA-3135") + log.warn(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 see KAFKA-3135") kafkaParams.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) } } From ec095fb7082b42e61136bcfafe0df72b7a453b95 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Mon, 18 Apr 2016 11:20:45 -0500 Subject: [PATCH 18/37] [SPARK-12177][Streaming][Kafka] code review changes --- external/kafka-beta-assembly/pom.xml | 2 +- .../streaming/kafka/CachedKafkaConsumer.scala | 21 +++++++++---------- .../kafka/DirectKafkaInputDStream.scala | 2 +- .../spark/streaming/kafka/KafkaRDD.scala | 2 +- 4 files changed, 13 insertions(+), 14 deletions(-) diff --git a/external/kafka-beta-assembly/pom.xml b/external/kafka-beta-assembly/pom.xml index 139f5d348768f..8fa1b689877b0 100644 --- a/external/kafka-beta-assembly/pom.xml +++ b/external/kafka-beta-assembly/pom.xml @@ -28,7 +28,7 @@ org.apache.spark spark-streaming-kafka-beta-assembly_2.11 jar - Spark Project External Kafka Assembly + Spark Project External Kafka Beta Assembly http://spark.apache.org/ diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index b0b1e63c96d69..73efd632b7541 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -61,25 +61,24 @@ class CachedKafkaConsumer[K, V] private( * Sequential forward access will use buffers, but random access will be horribly inefficient. */ def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = { - log.debug(s"get $groupId $topic $partition nextOffset $nextOffset requested $offset") + log.debug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") if (offset != nextOffset) { - log.info(s"initial fetch for $groupId $topic $partition $offset") - buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator + log.info(s"Initial fetch for $groupId $topic $partition $offset") seek(offset) poll(timeout) } if (!buffer.hasNext()) { poll(timeout) } assert(buffer.hasNext(), - s"failed to get records for $groupId $topic $partition $offset after polling for $timeout") + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") var record = buffer.next() if (record.offset != offset) { - log.info(s"buffer miss for $groupId $topic $partition $offset") + log.info(s"Buffer miss for $groupId $topic $partition $offset") seek(offset) poll(timeout) assert(buffer.hasNext(), - s"failed to get records for $groupId $topic $partition $offset after polling for $timeout") + s"Failed to get records for $groupId $topic $partition $offset after polling for $timeout") record = buffer.next() assert(record.offset == offset, s"Got wrong record for $groupId $topic $partition even after seeking to offset $offset") @@ -90,14 +89,14 @@ class CachedKafkaConsumer[K, V] private( } private def seek(offset: Long): Unit = { - log.debug(s"seeking to $topicPartition $offset") + log.debug(s"Seeking to $topicPartition $offset") consumer.seek(topicPartition, offset) } private def poll(timeout: Long): Unit = { val p = consumer.poll(timeout) val r = p.records(topicPartition) - log.debug(s"polled ${p.partitions()} ${r.size}") + log.debug(s"Polled ${p.partitions()} ${r.size}") buffer = r.iterator } @@ -117,7 +116,7 @@ object CachedKafkaConsumer extends Logging { maxCapacity: Int, loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { if (null == cache) { - log.info(s"initializing cache $initialCapacity $maxCapacity $loadFactor") + log.info(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( initialCapacity, loadFactor, true) { override def removeEldestEntry( @@ -146,7 +145,7 @@ object CachedKafkaConsumer extends Logging { val k = CacheKey(groupId, topic, partition) val v = cache.get(k) if (null == v) { - log.info(s"cache miss for $k") + log.info(s"Cache miss for $k") log.debug(cache.keySet.toString) val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) cache.put(k, c) @@ -161,7 +160,7 @@ object CachedKafkaConsumer extends Logging { def remove(groupId: String, topic: String, partition: Int): Unit = CachedKafkaConsumer.synchronized { val k = CacheKey(groupId, topic, partition) - log.info(s"removing $k from cache") + log.info(s"Removing $k from cache") val v = cache.get(k) if (null != v) { v.consumer.close() diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 6da4ba029c206..ea8f22b18df70 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -102,7 +102,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") - private[streaming] override def name: String = s"Kafka direct stream [$id]" + private[streaming] override def name: String = s"Kafka beta direct stream [$id]" protected[streaming] override val checkpointData = new DirectKafkaInputDStreamCheckpointData diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 144a6d4ea7173..5ddf5b4b9ed22 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -59,7 +59,7 @@ class KafkaRDD[ assert(false == kafkaParams.get(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG).asInstanceOf[Boolean], - ConsumerConfig.AUTO_OFFSET_RESET_CONFIG + + ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG + " must be set to false for executor kafka params, else offsets may commit before processing") // TODO is it necessary to have separate configs for initial poll time vs ongoing poll time? From 544bf888984e20dadae852faa8ca1dd26fc416e7 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 6 May 2016 09:28:02 -0500 Subject: [PATCH 19/37] [SPARK-12177][Streaming][Kafka] prevent NPE --- .../spark/streaming/kafka/DirectKafkaInputDStream.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index ea8f22b18df70..252525b5f2eff 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -315,7 +315,11 @@ object DirectKafkaInputDStream extends Logging { executorKafkaParams: ju.Map[String, Object], driverConsumer: () => Consumer[K, V] ): DirectKafkaInputDStream[K, V] = { - val ph = new ju.HashMap[TopicPartition, String](preferredHosts) + val ph = if (preferredHosts == preferBrokers) { + preferredHosts + } else { + new ju.HashMap[TopicPartition, String](preferredHosts) + } val ekp = new ju.HashMap[String, Object](executorKafkaParams) KafkaRDD.fixKafkaParams(ekp) val cleaned = ssc.sparkContext.clean(driverConsumer) From 44d343496795013ec80338ec620dff1e81f5c715 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 6 May 2016 10:55:03 -0500 Subject: [PATCH 20/37] [SPARK-12177][Streaming][Kafka] update to kafka 0.10 RC3 --- external/kafka-beta/pom.xml | 2 +- .../spark/streaming/kafka/DirectKafkaInputDStream.scala | 6 +++--- .../org/apache/spark/streaming/kafka/KafkaTestUtils.scala | 2 +- .../spark/streaming/kafka/DirectKafkaStreamSuite.scala | 4 ++-- 4 files changed, 7 insertions(+), 7 deletions(-) diff --git a/external/kafka-beta/pom.xml b/external/kafka-beta/pom.xml index a14b94a9aa0e5..5b38d8c3dd20b 100644 --- a/external/kafka-beta/pom.xml +++ b/external/kafka-beta/pom.xml @@ -51,7 +51,7 @@ org.apache.kafka kafka_${scala.binary.version} - 0.9.0.1 + 0.10.0.0 com.sun.jmx diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 252525b5f2eff..a0efee1e3d446 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -163,9 +163,9 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( // make sure new partitions are reflected in currentOffsets val newPartitions = parts.diff(currentOffsets.keySet) currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap - newPartitions.foreach(tp => c.pause(tp)) + c.pause(newPartitions.asJava) - c.seekToEnd() + c.seekToEnd(currentOffsets.keySet.asJava) parts.map(tp => tp -> c.position(tp)).toMap } @@ -220,7 +220,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } // don't actually want to consume any messages, so pause all partitions - currentOffsets.keySet.foreach(tp => c.pause(tp)) + c.pause(currentOffsets.keySet.asJava) } override def stop(): Unit = this.synchronized { diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala index 3998987cecd5f..b047c5462b588 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -107,7 +107,7 @@ private[kafka] class KafkaTestUtils extends Logging { // Kafka broker startup Utils.startServiceOnPort(brokerPort, port => { brokerPort = port - brokerConf = new KafkaConfig(brokerConfiguration) + brokerConf = new KafkaConfig(brokerConfiguration, doLog = false) server = new KafkaServer(brokerConf) server.startup() (server, port) diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index ffc647b341a5d..6e34a5389bbb5 100644 --- a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -165,7 +165,7 @@ class DirectKafkaStreamSuite val kc = new KafkaConsumer(kafkaParams) kc.assign(Arrays.asList(topicPartition)) def getLatestOffset(): Long = { - kc.seekToEnd(topicPartition) + kc.seekToEnd(Arrays.asList(topicPartition)) kc.position(topicPartition) } @@ -216,7 +216,7 @@ class DirectKafkaStreamSuite val kc = new KafkaConsumer(kafkaParams) kc.assign(Arrays.asList(topicPartition)) def getLatestOffset(): Long = { - kc.seekToEnd(topicPartition) + kc.seekToEnd(Arrays.asList(topicPartition)) kc.position(topicPartition) } From 7cf54957c746698d25b2adf5ed9ea324ff6f921f Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Fri, 27 May 2016 10:43:11 -0500 Subject: [PATCH 21/37] [SPARK-12177][Streaming][Kafka] exclude new kafka beta from mima --- project/SparkBuild.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 250074c985544..f568276a6917a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -340,7 +340,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, sketch, mllibLocal + unsafe, tags, sketch, mllibLocal, streamingKafkaBeta ).contains(x) } From 82067c047c371d33435f2d682456ee4525ec9e0a Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 14 Jun 2016 16:29:07 -0500 Subject: [PATCH 22/37] [SPARK-12177][Streaming][Kafka] rename kafka-beta to kafka-0-10 --- .../pom.xml | 8 ++++---- external/{kafka-beta => kafka-0-10}/pom.xml | 4 ++-- .../spark/streaming/kafka/CachedKafkaConsumer.scala | 0 .../streaming/kafka/DirectKafkaInputDStream.scala | 2 +- .../org/apache/spark/streaming/kafka/KafkaRDD.scala | 0 .../spark/streaming/kafka/KafkaRDDPartition.scala | 0 .../spark/streaming/kafka/KafkaTestUtils.scala | 0 .../apache/spark/streaming/kafka/OffsetRange.scala | 0 .../apache/spark/streaming/kafka/package-info.java | 0 .../org/apache/spark/streaming/kafka/package.scala | 0 .../streaming/kafka/JavaDirectKafkaStreamSuite.java | 0 .../spark/streaming/kafka/JavaKafkaRDDSuite.java | 0 .../src/test/resources/log4j.properties | 0 .../streaming/kafka/DirectKafkaStreamSuite.scala | 0 .../apache/spark/streaming/kafka/KafkaRDDSuite.scala | 0 pom.xml | 4 ++-- project/SparkBuild.scala | 12 ++++++------ 17 files changed, 15 insertions(+), 15 deletions(-) rename external/{kafka-beta-assembly => kafka-0-10-assembly}/pom.xml (95%) rename external/{kafka-beta => kafka-0-10}/pom.xml (96%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala (100%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala (99%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala (100%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala (100%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala (100%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala (100%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/package-info.java (100%) rename external/{kafka-beta => kafka-0-10}/src/main/scala/org/apache/spark/streaming/kafka/package.scala (100%) rename external/{kafka-beta => kafka-0-10}/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java (100%) rename external/{kafka-beta => kafka-0-10}/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java (100%) rename external/{kafka-beta => kafka-0-10}/src/test/resources/log4j.properties (100%) rename external/{kafka-beta => kafka-0-10}/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala (100%) rename external/{kafka-beta => kafka-0-10}/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala (100%) diff --git a/external/kafka-beta-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml similarity index 95% rename from external/kafka-beta-assembly/pom.xml rename to external/kafka-0-10-assembly/pom.xml index 9fd59204fbe2e..f2468d1cbac7f 100644 --- a/external/kafka-beta-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -26,19 +26,19 @@ org.apache.spark - spark-streaming-kafka-beta-assembly_2.11 + spark-streaming-kafka-0-10-assembly_2.11 jar - Spark Project External Kafka Beta Assembly + Spark Integration for Kafka 0.10 Assembly http://spark.apache.org/ - streaming-kafka-beta-assembly + streaming-kafka-0-10-assembly org.apache.spark - spark-streaming-kafka-beta_${scala.binary.version} + spark-streaming-kafka-0-10_${scala.binary.version} ${project.version} diff --git a/external/kafka-beta/pom.xml b/external/kafka-0-10/pom.xml similarity index 96% rename from external/kafka-beta/pom.xml rename to external/kafka-0-10/pom.xml index 85a9e5a173b9c..50395f6d14453 100644 --- a/external/kafka-beta/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -26,9 +26,9 @@ org.apache.spark - spark-streaming-kafka-beta_2.11 + spark-streaming-kafka-0-10_2.11 - streaming-kafka-beta + streaming-kafka-0-10 jar Spark Integration for Kafka 0.10 diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala similarity index 100% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala similarity index 99% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index a0efee1e3d446..3cd518bdf1f95 100644 --- a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -102,7 +102,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } // Keep this consistent with how other streams are named (e.g. "Flume polling stream [2]") - private[streaming] override def name: String = s"Kafka beta direct stream [$id]" + private[streaming] override def name: String = s"Kafka 0.10 direct stream [$id]" protected[streaming] override val checkpointData = new DirectKafkaInputDStreamCheckpointData diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala similarity index 100% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala similarity index 100% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala similarity index 100% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala similarity index 100% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java similarity index 100% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package-info.java rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java diff --git a/external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala similarity index 100% rename from external/kafka-beta/src/main/scala/org/apache/spark/streaming/kafka/package.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala diff --git a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java similarity index 100% rename from external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java rename to external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java diff --git a/external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java similarity index 100% rename from external/kafka-beta/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java rename to external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java diff --git a/external/kafka-beta/src/test/resources/log4j.properties b/external/kafka-0-10/src/test/resources/log4j.properties similarity index 100% rename from external/kafka-beta/src/test/resources/log4j.properties rename to external/kafka-0-10/src/test/resources/log4j.properties diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala similarity index 100% rename from external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala rename to external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala diff --git a/external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala similarity index 100% rename from external/kafka-beta/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala rename to external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala diff --git a/pom.xml b/pom.xml index d214a9a707b27..010afc03a61f3 100644 --- a/pom.xml +++ b/pom.xml @@ -109,8 +109,8 @@ launcher external/kafka-0-8 external/kafka-0-8-assembly - external/kafka-beta - external/kafka-beta-assembly + external/kafka-0-10 + external/kafka-0-10-assembly diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f568276a6917a..9bd6668527cd3 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -44,9 +44,9 @@ object BuildCommons { ).map(ProjectRef(buildLocation, _)) val streamingProjects@Seq( - streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingKafkaBeta + streaming, streamingFlumeSink, streamingFlume, streamingKafka, streamingKafka010 ) = Seq( - "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka-0-8", "streaming-kafka-beta" + "streaming", "streaming-flume-sink", "streaming-flume", "streaming-kafka-0-8", "streaming-kafka-0-10" ).map(ProjectRef(buildLocation, _)) val allProjects@Seq( @@ -61,8 +61,8 @@ object BuildCommons { Seq("yarn", "java8-tests", "ganglia-lgpl", "streaming-kinesis-asl", "docker-integration-tests").map(ProjectRef(buildLocation, _)) - val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafkaBetaAssembly, streamingKinesisAslAssembly) = - Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kafka-beta-assembly", "streaming-kinesis-asl-assembly") + val assemblyProjects@Seq(networkYarn, streamingFlumeAssembly, streamingKafkaAssembly, streamingKafka010Assembly, streamingKinesisAslAssembly) = + Seq("network-yarn", "streaming-flume-assembly", "streaming-kafka-0-8-assembly", "streaming-kafka-0-10-assembly", "streaming-kinesis-asl-assembly") .map(ProjectRef(buildLocation, _)) val copyJarsProjects@Seq(assembly, examples) = Seq("assembly", "examples") @@ -340,7 +340,7 @@ object SparkBuild extends PomBuild { val mimaProjects = allProjects.filterNot { x => Seq( spark, hive, hiveThriftServer, catalyst, repl, networkCommon, networkShuffle, networkYarn, - unsafe, tags, sketch, mllibLocal, streamingKafkaBeta + unsafe, tags, sketch, mllibLocal, streamingKafka010 ).contains(x) } @@ -581,7 +581,7 @@ object Assembly { .getOrElse(SbtPomKeys.effectivePom.value.getProperties.get("hadoop.version").asInstanceOf[String]) }, jarName in assembly <<= (version, moduleName, hadoopVersion) map { (v, mName, hv) => - if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-0-8-assembly") || mName.contains("streaming-kafka-beta-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { + if (mName.contains("streaming-flume-assembly") || mName.contains("streaming-kafka-0-8-assembly") || mName.contains("streaming-kafka-0-10-assembly") || mName.contains("streaming-kinesis-asl-assembly")) { // This must match the same name used in maven (see external/kafka-0-8-assembly/pom.xml) s"${mName}-${v}.jar" } else { From c7158724a39649a0f8d84e0e02e4d71f58a4f433 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 14 Jun 2016 23:12:34 -0500 Subject: [PATCH 23/37] [SPARK-12177][Streaming][Kafka] make test less flaky --- .../kafka/DirectKafkaStreamSuite.scala | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 6e34a5389bbb5..4fb29623bd379 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -264,13 +264,10 @@ class DirectKafkaStreamSuite val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") - // Send data to Kafka and wait for it to be received - def sendDataAndWaitForReceive(data: Seq[Int]) { + // Send data to Kafka + def sendData(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 @@ -288,21 +285,22 @@ class DirectKafkaStreamSuite } ssc.checkpoint(testDir.getAbsolutePath) - // This is to collect the raw data received from Kafka - kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => - val data = rdd.map { _.value }.collect() - DirectKafkaStreamSuite.collectedData.addAll(Arrays.asList(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 } + rdd.collect().headOption.foreach { x => + DirectKafkaStreamSuite.total.set(x._2) + } } + ssc.start() - // Send some data and wait for them to be received + // Send some data for (i <- (1 to 10).grouped(4)) { - sendDataAndWaitForReceive(i) + sendData(i) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) } // Verify that offset ranges were generated @@ -333,9 +331,12 @@ class DirectKafkaStreamSuite // 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) + for (i <- (11 to 20).grouped(4)) { + sendData(i) + } + eventually(timeout(10 seconds), interval(50 milliseconds)) { - assert(DirectKafkaStreamSuite.total === (1 to 20).sum) + assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) } ssc.stop() } @@ -582,8 +583,7 @@ class DirectKafkaStreamSuite } object DirectKafkaStreamSuite { - val collectedData = new ConcurrentLinkedQueue[String]() - @volatile var total = -1L + val total = new AtomicLong(-1L) class InputInfoCollector extends StreamingListener { val numRecordsSubmitted = new AtomicLong(0L) From ed6df9da5d7d211cc41e00aedd9a11088d948069 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 15 Jun 2016 09:37:16 -0500 Subject: [PATCH 24/37] [SPARK-12177][Streaming][Kafka] better test error messages --- .../spark/streaming/kafka/DirectKafkaStreamSuite.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 4fb29623bd379..6dbda3aeef6c4 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -319,14 +319,15 @@ class DirectKafkaStreamSuite ssc.graph.getInputStreams().head.asInstanceOf[DStream[ConsumerRecord[String, String]]] // Verify offset ranges have been recovered - val recoveredOffsetRanges = getOffsetRanges(recoveredStream) + val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => (x._1, x._2.toSet) } assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") - val earlierOffsetRangesAsSets = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } + val earlierOffsetRanges = offsetRangesBeforeStop.map { x => (x._1, x._2.toSet) } assert( recoveredOffsetRanges.forall { or => - earlierOffsetRangesAsSets.contains((or._1, or._2.toSet)) + earlierOffsetRanges.contains((or._1, or._2)) }, - "Recovered ranges are not the same as the ones generated" + "Recovered ranges are not the same as the ones generated\n" + + earlierOffsetRanges + "\n" + recoveredOffsetRanges ) // 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 From 3a95969db06cbff3eabfdde5ccf62ebbe85f967f Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 15 Jun 2016 10:11:49 -0500 Subject: [PATCH 25/37] [SPARK-12177][Streaming][Kafka] more tweaks to flaky test --- .../apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 6dbda3aeef6c4..6f50bd798d2ef 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -303,6 +303,8 @@ class DirectKafkaStreamSuite assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) } + ssc.stop() + // Verify that offset ranges were generated val offsetRangesBeforeStop = getOffsetRanges(kafkaStream) assert(offsetRangesBeforeStop.size >= 1, "No offset ranges generated") @@ -310,7 +312,7 @@ class DirectKafkaStreamSuite offsetRangesBeforeStop.head._2.forall { _.fromOffset === 0 }, "starting offset not zero" ) - ssc.stop() + logInfo("====== RESTARTING ========") // Recover context from checkpoints From 9dd7bcea907c1fd69b4a0e121ff1764f480b7d97 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 15 Jun 2016 11:40:58 -0500 Subject: [PATCH 26/37] [SPARK-12177][Streaming][Kafka] doc cleanup, add experimental annotation --- .../kafka/DirectKafkaInputDStream.scala | 60 ++++++++++++++++--- .../spark/streaming/kafka/KafkaRDD.scala | 40 ++++++++++++- .../spark/streaming/kafka/OffsetRange.scala | 2 +- 3 files changed, 93 insertions(+), 9 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 3cd518bdf1f95..c71c219b43206 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -29,6 +29,7 @@ import scala.reflect.ClassTag import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.{ PartitionInfo, TopicPartition } +import org.apache.spark.annotation.Experimental import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel @@ -43,15 +44,22 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator * 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 + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] + * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @param executorKafkaParams Kafka + * * configuration parameters. * Requires "bootstrap.servers" to be set with Kafka broker(s), * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, + * and subscribe topics or assign partitions. + * This consumer will be used on the driver to query for offsets only, not messages. + * See Consumer doc + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value */ - +@Experimental class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( _ssc: StreamingContext, preferredHosts: ju.Map[TopicPartition, String], @@ -299,6 +307,10 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } } +/** + * Companion object that provides methods to create instances of [[DirectKafkaInputDStream]] + */ +@Experimental object DirectKafkaInputDStream extends Logging { import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } import org.apache.spark.api.java.function.{ Function0 => JFunction0 } @@ -308,7 +320,23 @@ object DirectKafkaInputDStream extends Logging { /** Prefer a consistent executor per TopicPartition, evenly from all executors */ val preferConsistent: ju.Map[TopicPartition, String] = ju.Collections.emptyMap() - /** Scala constructor */ + /** + * Scala constructor + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] + * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @param executorKafkaParams Kafka + * + * configuration parameters. + * Requires "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, + * and subscribe topics or assign partitions. + * This consumer will be used on the driver to query for offsets only, not messages. + * See Consumer doc + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ def apply[K: ClassTag, V: ClassTag]( ssc: StreamingContext, preferredHosts: ju.Map[TopicPartition, String], @@ -327,7 +355,25 @@ object DirectKafkaInputDStream extends Logging { new DirectKafkaInputDStream[K, V](ssc, ph, ekp, cleaned) } - /** Java constructor */ + /** + * Java constructor + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] + * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @param executorKafkaParams Kafka + * + * configuration parameters. + * Requires "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, + * and subscribe topics or assign partitions. + * This consumer will be used on the driver to query for offsets only, not messages. + * See Consumer doc + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ def create[K, V]( jssc: JavaStreamingContext, keyClass: Class[K], diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 5ddf5b4b9ed22..bcf588b0f29bb 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -26,6 +26,7 @@ import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} +import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD @@ -41,8 +42,13 @@ import org.apache.spark.storage.StorageLevel * 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 + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] + * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value */ - +@Experimental class KafkaRDD[ K: ClassTag, V: ClassTag] private[spark] ( @@ -201,6 +207,10 @@ class KafkaRDD[ } } +/** + * Companion object that provides methods to create instances of [[KafkaRDD]] + */ +@Experimental object KafkaRDD extends Logging { import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } @@ -224,6 +234,19 @@ object KafkaRDD extends Logging { } } + /** + * Scala constructor + * @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 + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] + * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ def apply[K: ClassTag, V: ClassTag]( sc: SparkContext, kafkaParams: ju.Map[String, Object], @@ -242,6 +265,21 @@ object KafkaRDD extends Logging { new KafkaRDD[K, V](sc, kp, osr, ph) } + /** + * Java constructor + * @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) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] + * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ def create[K, V]( jsc: JavaSparkContext, keyClass: Class[K], diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index 7ad965ee9cc96..0732f77cea863 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -22,7 +22,7 @@ import org.apache.kafka.common.TopicPartition /** * 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 - * [[DirectKafkaInputDStream()]]). + * [[DirectKafkaInputDStream]]). * {{{ * DirectKafkaInputDStream(...).foreachRDD { rdd => * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges From b4043049a9e864bf0b0c0e13affe5102c6c9278c Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 15 Jun 2016 16:29:17 -0500 Subject: [PATCH 27/37] [SPARK-12177][Streaming][Kafka] when recovering from checkpoint, disable consumer cache --- .../streaming/kafka/CachedKafkaConsumer.scala | 18 ++++++++++++++++-- .../kafka/DirectKafkaInputDStream.scala | 11 +++++++++-- .../spark/streaming/kafka/KafkaRDD.scala | 18 +++++++++++++++--- 3 files changed, 40 insertions(+), 7 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index 73efd632b7541..476dfc6b530b6 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -29,7 +29,7 @@ import org.apache.spark.internal.Logging /** * Consumer of single topicpartition, intended for cached reuse. * Underlying consumer is not threadsafe, so neither is this, - * but processing the same topicpartition and group id in multiple threads would be bad anyway. + * but processing the same topicpartition and group id in multiple threads is usually bad anyway. */ private[kafka] class CachedKafkaConsumer[K, V] private( @@ -56,6 +56,8 @@ class CachedKafkaConsumer[K, V] private( protected var buffer = ju.Collections.emptyList[ConsumerRecord[K, V]]().iterator protected var nextOffset = -2L + def close(): Unit = consumer.close() + /** * Get the record for the given offset, waiting up to timeout ms if IO is necessary. * Sequential forward access will use buffers, but random access will be horribly inefficient. @@ -156,6 +158,17 @@ object CachedKafkaConsumer extends Logging { } } + /** + * Get a fresh new instance, unassociated with the global cache. + * Caller is responsible for closing + */ + def getUncached[K, V]( + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) + /** remove consumer for given groupId, topic, and partition, if it exists */ def remove(groupId: String, topic: String, partition: Int): Unit = CachedKafkaConsumer.synchronized { @@ -163,8 +176,9 @@ object CachedKafkaConsumer extends Logging { log.info(s"Removing $k from cache") val v = cache.get(k) if (null != v) { - v.consumer.close() + v.close() cache.remove(k) + log.info(s"Removed $k from cache") } } } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index c71c219b43206..4d3c007f9cb47 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -196,7 +196,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( OffsetRange(tp.topic, tp.partition, fo, uo) } val rdd = new KafkaRDD[K, V]( - context.sparkContext, executorKafkaParams, offsetRanges.toArray, getPreferredHosts) + context.sparkContext, executorKafkaParams, offsetRanges.toArray, getPreferredHosts, true) // Report the record number and metadata of this batch interval to InputInfoTracker. val description = offsetRanges.filter { offsetRange => @@ -293,7 +293,14 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( 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]( - context.sparkContext, executorKafkaParams, b.map(OffsetRange(_)), getPreferredHosts) + context.sparkContext, + executorKafkaParams, + b.map(OffsetRange(_)), + getPreferredHosts, + // during restore, it's possible same partition will be consumed from multiple + // threads, so dont use cache + false + ) } } } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index bcf588b0f29bb..f3f905d2c7380 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -45,6 +45,7 @@ import org.apache.spark.storage.StorageLevel * @param preferredHosts map from TopicPartition to preferred host for processing that partition. * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @param useConsumerCache whether to use a consumer from a per-jvm cache * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -55,7 +56,8 @@ class KafkaRDD[ sc: SparkContext, val kafkaParams: ju.Map[String, Object], val offsetRanges: Array[OffsetRange], - val preferredHosts: ju.Map[TopicPartition, String] + val preferredHosts: ju.Map[TopicPartition, String], + useConsumerCache: Boolean ) extends RDD[ConsumerRecord[K, V]](sc, Nil) with Logging with HasOffsetRanges { assert("none" == @@ -185,17 +187,27 @@ class KafkaRDD[ val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] - val consumer = { + context.addTaskCompletionListener{ context => closeIfNeeded() } + + val consumer = if (useConsumerCache) { CachedKafkaConsumer.init(cacheInitialCapacity, cacheMaxCapacity, cacheLoadFactor) if (context.attemptNumber > 1) { // just in case the prior attempt failures were cache related CachedKafkaConsumer.remove(groupId, part.topic, part.partition) } CachedKafkaConsumer.get[K, V](groupId, part.topic, part.partition, kafkaParams) + } else { + CachedKafkaConsumer.getUncached[K, V](groupId, part.topic, part.partition, kafkaParams) } var requestOffset = part.fromOffset + def closeIfNeeded(): Unit = { + if (!useConsumerCache && consumer != null) { + consumer.close + } + } + override def hasNext(): Boolean = requestOffset < part.untilOffset override def next(): ConsumerRecord[K, V] = { @@ -262,7 +274,7 @@ object KafkaRDD extends Logging { val osr = offsetRanges.clone() val ph = new ju.HashMap[TopicPartition, String](preferredHosts) - new KafkaRDD[K, V](sc, kp, osr, ph) + new KafkaRDD[K, V](sc, kp, osr, ph, true) } /** From 0f15bd138b7cdb5468d5515a1b0c02b09c60136a Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Sat, 25 Jun 2016 23:28:51 -0500 Subject: [PATCH 28/37] [SPARK-12177][Streaming][Kafka] code + doc cleanup --- .../streaming/kafka/CachedKafkaConsumer.scala | 22 ++++++++-------- .../kafka/DirectKafkaInputDStream.scala | 10 ++++---- .../spark/streaming/kafka/KafkaRDD.scala | 25 +++++++++++-------- 3 files changed, 31 insertions(+), 26 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index 476dfc6b530b6..5837d6f1e930f 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -114,9 +114,9 @@ object CachedKafkaConsumer extends Logging { /** Must be called before get, once per JVM, to configure the cache. Further calls are ignored */ def init( - initialCapacity: Int, - maxCapacity: Int, - loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { + initialCapacity: Int, + maxCapacity: Int, + loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { if (null == cache) { log.info(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( @@ -139,10 +139,10 @@ object CachedKafkaConsumer extends Logging { * If matching consumer doesn't already exist, will be created using kafkaParams. */ def get[K, V]( - groupId: String, - topic: String, - partition: Int, - kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = CachedKafkaConsumer.synchronized { val k = CacheKey(groupId, topic, partition) val v = cache.get(k) @@ -163,10 +163,10 @@ object CachedKafkaConsumer extends Logging { * Caller is responsible for closing */ def getUncached[K, V]( - groupId: String, - topic: String, - partition: Int, - kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = + groupId: String, + topic: String, + partition: Int, + kafkaParams: ju.Map[String, Object]): CachedKafkaConsumer[K, V] = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) /** remove consumer for given groupId, topic, and partition, if it exists */ diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 4d3c007f9cb47..f7a19188b0e7c 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -39,7 +39,7 @@ import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} import org.apache.spark.streaming.scheduler.rate.RateEstimator /** - * A stream of {@link org.apache.spark.streaming.kafka.KafkaRDD} where + * A stream of [[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 @@ -77,7 +77,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( consumer() override def persist(newLevel: StorageLevel): DStream[ConsumerRecord[K, V]] = { - log.error("Kafka ConsumerRecord is not serializable. " + + logError("Kafka ConsumerRecord is not serializable. " + "Use .map to extract fields before calling .persist or .window") super.persist(newLevel) } @@ -279,7 +279,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] } - override def update(time: Time) { + override def update(time: Time): Unit = { batchForTime.clear() generatedRDDs.foreach { kv => val a = kv._2.asInstanceOf[KafkaRDD[K, V]].offsetRanges.map(_.toTuple).toArray @@ -287,9 +287,9 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } } - override def cleanup(time: Time) { } + override def cleanup(time: Time): Unit = { } - override def restore() { + override def restore(): Unit = { 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]( diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index f3f905d2c7380..a9cc616f7067c 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -80,7 +80,7 @@ class KafkaRDD[ conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat override def persist(newLevel: StorageLevel): this.type = { - log.error("Kafka ConsumerRecord is not serializable. " + + logError("Kafka ConsumerRecord is not serializable. " + "Use .map to extract fields before calling .persist or .window") super.persist(newLevel) } @@ -108,7 +108,7 @@ class KafkaRDD[ .map(_.asInstanceOf[KafkaRDDPartition]) .filter(_.count > 0) - if (num < 1 || nonEmptyPartitions.size < 1) { + if (num < 1 || nonEmptyPartitions.isEmpty) { return new Array[ConsumerRecord[K, V]](0) } @@ -140,7 +140,9 @@ class KafkaRDD[ .sortWith((a, b) => a.host > b.host || a.executorId > b.executorId) } - // non-negative modulus, from java 8 math + /** + * Non-negative modulus, from java 8 math + */ private def floorMod(a: Int, b: Int): Int = ((a % b) + b) % b override def getPreferredLocations(thePart: Partition): Seq[String] = { @@ -156,7 +158,6 @@ class KafkaRDD[ } else { val index = this.floorMod(tp.hashCode, execs.length) val chosen = execs(index) - Seq(chosen.toString) } } @@ -170,7 +171,7 @@ class KafkaRDD[ 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 " + + logInfo(s"Beginning offset ${part.fromOffset} is the same as ending offset " + s"skipping ${part.topic} ${part.partition}") Iterator.empty } else { @@ -178,11 +179,15 @@ class KafkaRDD[ } } + /** + * An iterator that fetches messages directly from Kafka for the offsets in partition. + * Uses a cached consumer where possible to take advantage of prefetching + */ private class KafkaRDDIterator( part: KafkaRDDPartition, context: TaskContext) extends Iterator[ConsumerRecord[K, V]] { - log.info(s"Computing topic ${part.topic}, partition ${part.partition} " + + logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + s"offsets ${part.fromOffset} -> ${part.untilOffset}") val groupId = kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG).asInstanceOf[String] @@ -227,21 +232,21 @@ object KafkaRDD extends Logging { import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } private[kafka] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { - log.warn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") + logWarn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) - log.warn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") + logWarn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") // driver and executor should be in different consumer groups val groupId = "spark-executor-" + kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) - log.warn(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") + logWarn(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) // possible workaround for KAFKA-3135 val rbb = kafkaParams.get(ConsumerConfig.RECEIVE_BUFFER_CONFIG) if (null == rbb || rbb.asInstanceOf[java.lang.Integer] < 65536) { - log.warn(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 see KAFKA-3135") + logWarn(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 see KAFKA-3135") kafkaParams.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) } } From 0610e9b26be6558fc68654b9749850ede70666ef Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Sun, 26 Jun 2016 01:21:34 -0500 Subject: [PATCH 29/37] [SPARK-12177][Streaming][Kafka] hide DirectKafkaInputDStream and KafkaRDD --- .../kafka/DirectKafkaInputDStream.scala | 92 +------- .../spark/streaming/kafka/KafkaRDD.scala | 96 +------- .../spark/streaming/kafka/KafkaUtils.scala | 211 ++++++++++++++++++ .../spark/streaming/kafka/OffsetRange.scala | 38 +++- .../kafka/JavaDirectKafkaStreamSuite.java | 6 +- .../streaming/kafka/JavaKafkaRDDSuite.java | 4 +- .../kafka/DirectKafkaStreamSuite.scala | 66 +++--- .../spark/streaming/kafka/KafkaRDDSuite.scala | 13 +- 8 files changed, 302 insertions(+), 224 deletions(-) create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index f7a19188b0e7c..47c6f7a494b0d 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -60,12 +60,12 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator * @tparam V type of Kafka message value */ @Experimental -class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( +private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( _ssc: StreamingContext, preferredHosts: ju.Map[TopicPartition, String], executorKafkaParams: ju.Map[String, Object], driverConsumer: () => Consumer[K, V] - ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging { + ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging with CanCommitOffsets { @transient private var kc: Consumer[K, V] = null def consumer(): Consumer[K, V] = this.synchronized { @@ -102,7 +102,7 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( } protected def getPreferredHosts: ju.Map[TopicPartition, String] = { - if (preferredHosts == DirectKafkaInputDStream.preferBrokers) { + if (preferredHosts == KafkaUtils.preferBrokers) { getBrokers } else { preferredHosts @@ -313,89 +313,3 @@ class DirectKafkaInputDStream[K: ClassTag, V: ClassTag] private[spark] ( override def publish(rate: Long): Unit = () } } - -/** - * Companion object that provides methods to create instances of [[DirectKafkaInputDStream]] - */ -@Experimental -object DirectKafkaInputDStream extends Logging { - import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } - import org.apache.spark.api.java.function.{ Function0 => JFunction0 } - - /** Prefer to run on kafka brokers, if they are on same hosts as executors */ - val preferBrokers: ju.Map[TopicPartition, String] = null - /** Prefer a consistent executor per TopicPartition, evenly from all executors */ - val preferConsistent: ju.Map[TopicPartition, String] = ju.Collections.emptyMap() - - /** - * Scala constructor - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] - * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. - * @param executorKafkaParams Kafka - * - * configuration parameters. - * Requires "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, - * and subscribe topics or assign partitions. - * This consumer will be used on the driver to query for offsets only, not messages. - * See Consumer doc - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - */ - def apply[K: ClassTag, V: ClassTag]( - ssc: StreamingContext, - preferredHosts: ju.Map[TopicPartition, String], - executorKafkaParams: ju.Map[String, Object], - driverConsumer: () => Consumer[K, V] - ): DirectKafkaInputDStream[K, V] = { - val ph = if (preferredHosts == preferBrokers) { - preferredHosts - } else { - new ju.HashMap[TopicPartition, String](preferredHosts) - } - val ekp = new ju.HashMap[String, Object](executorKafkaParams) - KafkaRDD.fixKafkaParams(ekp) - val cleaned = ssc.sparkContext.clean(driverConsumer) - - new DirectKafkaInputDStream[K, V](ssc, ph, ekp, cleaned) - } - - /** - * Java constructor - * @param keyClass Class of the keys in the Kafka records - * @param valueClass Class of the values in the Kafka records - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] - * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. - * @param executorKafkaParams Kafka - * - * configuration parameters. - * Requires "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, - * and subscribe topics or assign partitions. - * This consumer will be used on the driver to query for offsets only, not messages. - * See Consumer doc - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - */ - def create[K, V]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - preferredHosts: ju.Map[TopicPartition, String], - executorKafkaParams: ju.Map[String, Object], - driverConsumer: JFunction0[Consumer[K, V]] - ): JavaInputDStream[ConsumerRecord[K, V]] = { - - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - - new JavaInputDStream( - DirectKafkaInputDStream[K, V]( - jssc.ssc, preferredHosts, executorKafkaParams, driverConsumer.call _)) - } - -} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index a9cc616f7067c..4c87a3af74881 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -50,9 +50,9 @@ import org.apache.spark.storage.StorageLevel * @tparam V type of Kafka message value */ @Experimental -class KafkaRDD[ +private[spark] class KafkaRDD[ K: ClassTag, - V: ClassTag] private[spark] ( + V: ClassTag]( sc: SparkContext, val kafkaParams: ju.Map[String, Object], val offsetRanges: Array[OffsetRange], @@ -146,6 +146,8 @@ class KafkaRDD[ private def floorMod(a: Int, b: Int): Int = ((a % b) + b) % b override def getPreferredLocations(thePart: Partition): Seq[String] = { + // The intention is best-effort consistent executor for a given topicpartition, + // so that caching consumers can be effective. // TODO what about hosts specified by ip vs name val part = thePart.asInstanceOf[KafkaRDDPartition] val allExecs = executors() @@ -156,6 +158,7 @@ class KafkaRDD[ if (execs.isEmpty) { Seq() } else { + // execs is sorted, tp.hashCode depends only on topic and partition, so consistent index val index = this.floorMod(tp.hashCode, execs.length) val chosen = execs(index) Seq(chosen.toString) @@ -223,92 +226,3 @@ class KafkaRDD[ } } } - -/** - * Companion object that provides methods to create instances of [[KafkaRDD]] - */ -@Experimental -object KafkaRDD extends Logging { - import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } - - private[kafka] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { - logWarn(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") - kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) - - logWarn(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") - kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") - - // driver and executor should be in different consumer groups - val groupId = "spark-executor-" + kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) - logWarn(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") - kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) - - // possible workaround for KAFKA-3135 - val rbb = kafkaParams.get(ConsumerConfig.RECEIVE_BUFFER_CONFIG) - if (null == rbb || rbb.asInstanceOf[java.lang.Integer] < 65536) { - logWarn(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 see KAFKA-3135") - kafkaParams.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) - } - } - - /** - * Scala constructor - * @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 - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] - * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - */ - def apply[K: ClassTag, V: ClassTag]( - sc: SparkContext, - kafkaParams: ju.Map[String, Object], - offsetRanges: Array[OffsetRange], - preferredHosts: ju.Map[TopicPartition, String] - ): KafkaRDD[K, V] = { - assert(preferredHosts != DirectKafkaInputDStream.preferBrokers, - "If you want to prefer brokers, you must provide a mapping for preferredHosts. " + - "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you. ") - - val kp = new ju.HashMap[String, Object](kafkaParams) - fixKafkaParams(kp) - val osr = offsetRanges.clone() - val ph = new ju.HashMap[TopicPartition, String](preferredHosts) - - new KafkaRDD[K, V](sc, kp, osr, ph, true) - } - - /** - * Java constructor - * @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) specified in host1:port1,host2:port2 form. - * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] - * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - */ - def create[K, V]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - kafkaParams: ju.Map[String, Object], - offsetRanges: Array[OffsetRange], - preferredHosts: ju.Map[TopicPartition, String] - ): JavaRDD[ConsumerRecord[K, V]] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - - new JavaRDD(KafkaRDD[K, V](jsc.sc, kafkaParams, offsetRanges, preferredHosts)) - } - -} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala new file mode 100644 index 0000000000000..0761b87ae32a1 --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -0,0 +1,211 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.{ util => ju } + +import scala.reflect.{classTag, ClassTag} + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } +import org.apache.spark.api.java.function.{ Function0 => JFunction0 } +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } +import org.apache.spark.streaming.dstream._ + +@Experimental +object KafkaUtils extends Logging { + /** + * Scala constructor for 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 + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[KafkaUtils.preferConsistent]] + * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createRDD[K: ClassTag, V: ClassTag]( + sc: SparkContext, + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + preferredHosts: ju.Map[TopicPartition, String] + ): RDD[ConsumerRecord[K, V]] = { + assert(preferredHosts != KafkaUtils.preferBrokers, + "If you want to prefer brokers, you must provide a mapping for preferredHosts. " + + "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you. ") + + val kp = new ju.HashMap[String, Object](kafkaParams) + fixKafkaParams(kp) + val osr = offsetRanges.clone() + val ph = new ju.HashMap[TopicPartition, String](preferredHosts) + + new KafkaRDD[K, V](sc, kp, osr, ph, true) + } + + /** + * Java constructor for 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 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) specified in host1:port1,host2:port2 form. + * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[KafkaUtils.preferConsistent]] + * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createRDD[K, V]( + jsc: JavaSparkContext, + keyClass: Class[K], + valueClass: Class[V], + kafkaParams: ju.Map[String, Object], + offsetRanges: Array[OffsetRange], + preferredHosts: ju.Map[TopicPartition, String] + ): JavaRDD[ConsumerRecord[K, V]] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + + new JavaRDD(createRDD[K, V](jsc.sc, kafkaParams, offsetRanges, preferredHosts)) + } + + /** Prefer to run on kafka brokers, if they are on same hosts as executors */ + val preferBrokers: ju.Map[TopicPartition, String] = null + /** Prefer a consistent executor per TopicPartition, evenly from all executors */ + val preferConsistent: ju.Map[TopicPartition, String] = ju.Collections.emptyMap() + + /** + * Scala constructor for a DStream 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. + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[KafkaUtils.preferConsistent]] + * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. + * @param executorKafkaParams Kafka + * + * configuration parameters. + * Requires "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, + * and subscribe topics or assign partitions. + * This consumer will be used on the driver to query for offsets only, not messages. + * See Consumer doc + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K: ClassTag, V: ClassTag]( + ssc: StreamingContext, + preferredHosts: ju.Map[TopicPartition, String], + executorKafkaParams: ju.Map[String, Object], + driverConsumer: () => Consumer[K, V] + ): InputDStream[ConsumerRecord[K, V]] = { + val ph = if (preferredHosts == preferBrokers) { + preferredHosts + } else { + new ju.HashMap[TopicPartition, String](preferredHosts) + } + val ekp = new ju.HashMap[String, Object](executorKafkaParams) + fixKafkaParams(ekp) + val cleaned = ssc.sparkContext.clean(driverConsumer) + + new DirectKafkaInputDStream[K, V](ssc, ph, ekp, cleaned) + } + + /** + * Java constructor for a DStream where + * each given Kafka topic/partition corresponds to an RDD partition. + * @param keyClass Class of the keys in the Kafka records + * @param valueClass Class of the values in the Kafka records + * @param preferredHosts map from TopicPartition to preferred host for processing that partition. + * In most cases, use [[KafkaUtils.preferConsistent]] + * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. + * @param executorKafkaParams Kafka + * + * configuration parameters. + * Requires "bootstrap.servers" to be set with Kafka broker(s), + * NOT zookeeper servers, specified in host1:port1,host2:port2 form. + * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, + * and subscribe topics or assign partitions. + * This consumer will be used on the driver to query for offsets only, not messages. + * See Consumer doc + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ + @Experimental + def createDirectStream[K, V]( + jssc: JavaStreamingContext, + keyClass: Class[K], + valueClass: Class[V], + preferredHosts: ju.Map[TopicPartition, String], + executorKafkaParams: ju.Map[String, Object], + driverConsumer: JFunction0[Consumer[K, V]] + ): JavaInputDStream[ConsumerRecord[K, V]] = { + + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + + new JavaInputDStream( + createDirectStream[K, V]( + jssc.ssc, preferredHosts, executorKafkaParams, driverConsumer.call _)) + } + + + /** + * Tweak kafka params to prevent issues on executors + */ + private[kafka] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { + logWarning(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") + kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) + + logWarning(s"overriding ${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG} to none for executor") + kafkaParams.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") + + // driver and executor should be in different consumer groups + val groupId = "spark-executor-" + kafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) + logWarning(s"overriding executor ${ConsumerConfig.GROUP_ID_CONFIG} to ${groupId}") + kafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, groupId) + + // possible workaround for KAFKA-3135 + val rbb = kafkaParams.get(ConsumerConfig.RECEIVE_BUFFER_CONFIG) + if (null == rbb || rbb.asInstanceOf[java.lang.Integer] < 65536) { + logWarning(s"overriding ${ConsumerConfig.RECEIVE_BUFFER_CONFIG} to 65536 see KAFKA-3135") + kafkaParams.put(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) + } + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index 0732f77cea863..66ef2ce35d70d 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -18,13 +18,14 @@ package org.apache.spark.streaming.kafka import org.apache.kafka.common.TopicPartition +import org.apache.kafka.clients.consumer.OffsetCommitCallback /** * 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 - * [[DirectKafkaInputDStream]]). + * [[KafkaUtils.createDirectStream]]). * {{{ - * DirectKafkaInputDStream(...).foreachRDD { rdd => + * KafkaUtils.createDirectStream(...).foreachRDD { rdd => * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges * ... * } @@ -34,6 +35,39 @@ trait HasOffsetRanges { def offsetRanges: Array[OffsetRange] } +/** + * Represents any object that can commit a collection of [[OffsetRange]]s. + * The direct Kafka DStream implements this interface (see + * [[KafkaUtils.createDirectStream]]). + * {{{ + * val stream = KafkaUtils.createDirectStream(...) + * ... + * stream.asInstanceOf[CanCommitOffsets].commitAsync(offsets, new OffsetCommitCallback() { + * def onComplete(m: java.util.Map[TopicPartition, OffsetAndMetadata], e: Exception) { + * if (null != e) { + * // error + * } else { + * // success + * } + * } + * }) + * }}} + */ +trait CanCommitOffsets { + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange]): Unit + + /** + * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. + * @param callback Only the most recently provided callback will be used at commit. + */ + def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit +} + /** * Represents a range of offsets from a single Kafka TopicPartition. Instances of this class * can be created with `OffsetRange.create()`. diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index 850101a40d67d..bc4c8c344a8ff 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -91,9 +91,9 @@ public void testKafkaStream() throws InterruptedException { kafkaParams.put("auto.offset.reset", "earliest"); kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); - Map preferredHosts = DirectKafkaInputDStream.preferConsistent(); + Map preferredHosts = KafkaUtils.preferConsistent(); - JavaInputDStream istream1 = DirectKafkaInputDStream.create( + JavaInputDStream istream1 = KafkaUtils.createDirectStream( ssc, String.class, String.class, @@ -135,7 +135,7 @@ public String call(ConsumerRecord r) { final Map kafkaParams2 = new HashMap<>(kafkaParams); kafkaParams2.put("group.id", "java-test-consumer-" + random.nextInt()); - JavaInputDStream istream2 = DirectKafkaInputDStream.create( + JavaInputDStream istream2 = KafkaUtils.createDirectStream( ssc, String.class, String.class, diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 6e02889c6d67b..1e66054f54fb4 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -93,7 +93,7 @@ public String call(ConsumerRecord r) { } }; - JavaRDD rdd1 = KafkaRDD.create( + JavaRDD rdd1 = KafkaUtils.createRDD( sc, String.class, String.class, @@ -102,7 +102,7 @@ public String call(ConsumerRecord r) { leaders ).map(handler); - JavaRDD rdd2 = KafkaRDD.create( + JavaRDD rdd2 = KafkaUtils.createRDD( sc, String.class, String.class, diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 6f50bd798d2ef..f016a8e6aa0dc 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -93,7 +93,7 @@ class DirectKafkaStreamSuite kp } - val preferredHosts = DirectKafkaInputDStream.preferConsistent + val preferredHosts = KafkaUtils.preferConsistent test("basic stream receiving with multiple topics and smallest starting offset") { val topics = List("basic1", "basic2", "basic3") @@ -107,7 +107,7 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + KafkaUtils.createDirectStream[String, String](ssc, preferredHosts, kafkaParams, () => { val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topics: _*)) consumer @@ -180,11 +180,12 @@ class DirectKafkaStreamSuite // Setup context and kafka stream with largest offset ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - val s = DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topic)) - consumer - }) + val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, + () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer + }) s.consumer.poll(0) assert( s.consumer.position(topicPartition) >= offsetBeforeStart, @@ -231,12 +232,13 @@ class DirectKafkaStreamSuite // Setup context and kafka stream with largest offset ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - val s = DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.assign(Arrays.asList(topicPartition)) - consumer.seek(topicPartition, 11L) - consumer - }) + val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, + () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.assign(Arrays.asList(topicPartition)) + consumer.seek(topicPartition, 11L) + consumer + }) s.consumer.poll(0) assert( s.consumer.position(topicPartition) >= offsetBeforeStart, @@ -273,7 +275,7 @@ class DirectKafkaStreamSuite // Setup the streaming context ssc = new StreamingContext(sparkConf, Milliseconds(100)) val kafkaStream = withClue("Error creating direct stream") { - DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + KafkaUtils.createDirectStream[String, String](ssc, preferredHosts, kafkaParams, () => { val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topic)) consumer @@ -369,7 +371,7 @@ class DirectKafkaStreamSuite // Setup the streaming context ssc = new StreamingContext(sparkConf, Milliseconds(100)) withClue("Error creating direct stream") { - val kafkaStream = DirectKafkaInputDStream[String, String]( + val kafkaStream = KafkaUtils.createDirectStream[String, String]( ssc, preferredHosts, kafkaParams, () => { val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topic)) @@ -379,15 +381,16 @@ class DirectKafkaStreamSuite val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges val data = rdd.map(_.value).collect() collectedData.addAll(Arrays.asList(data: _*)) - kafkaStream.commitAsync(offsets, new OffsetCommitCallback() { - def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { - if (null != e) { - logError("commit failed", e) - } else { - committed.putAll(m) + kafkaStream.asInstanceOf[CanCommitOffsets] + .commitAsync(offsets, new OffsetCommitCallback() { + def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { + if (null != e) { + logError("commit failed", e) + } else { + committed.putAll(m) + } } - } - }) + }) } } ssc.start() @@ -424,7 +427,7 @@ class DirectKafkaStreamSuite ssc.addStreamingListener(collector) val stream = withClue("Error creating direct stream") { - DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, () => { + KafkaUtils.createDirectStream[String, String](ssc, preferredHosts, kafkaParams, () => { val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topic)) consumer @@ -483,7 +486,7 @@ class DirectKafkaStreamSuite kafkaTestUtils.createTopic(topic, 2) val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") val executorKafkaParams = new JHashMap[String, Object](kafkaParams) - KafkaRDD.fixKafkaParams(executorKafkaParams) + KafkaUtils.fixKafkaParams(executorKafkaParams) val batchIntervalMilliseconds = 100 val estimator = new ConstantEstimator(100) @@ -501,11 +504,12 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) val kafkaStream = withClue("Error creating direct stream") { - new DirectKafkaInputDStream[String, String](ssc, preferredHosts, executorKafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topic)) - consumer - }) { + new DirectKafkaInputDStream[String, String](ssc, preferredHosts, executorKafkaParams, + () => { + val consumer = new KafkaConsumer[String, String](kafkaParams) + consumer.subscribe(Arrays.asList(topic)) + consumer + }) { override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) }.map(r => (r.key, r.value)) @@ -564,7 +568,7 @@ class DirectKafkaStreamSuite val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") val executorKafkaParams = new JHashMap[String, Object](kafkaParams) - KafkaRDD.fixKafkaParams(executorKafkaParams) + KafkaUtils.fixKafkaParams(executorKafkaParams) val s = new DirectKafkaInputDStream[String, String]( ssc, diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index a0eb8f8b01c64..babd8868f66b6 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -71,7 +71,8 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) - val rdd = KafkaRDD[String, String](sc, kafkaParams, offsetRanges, preferredHosts).map(_.value) + val rdd = KafkaUtils.createRDD[String, String](sc, kafkaParams, offsetRanges, preferredHosts) + .map(_.value) val received = rdd.collect.toSet assert(received === messages.toSet) @@ -84,7 +85,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { assert(rdd.take(1).head === messages.head) assert(rdd.take(messages.size + 10).size === messages.size) - val emptyRdd = KafkaRDD[String, String]( + val emptyRdd = KafkaUtils.createRDD[String, String]( sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0)), preferredHosts) assert(emptyRdd.isEmpty) @@ -92,7 +93,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { // invalid offset ranges throw exceptions val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) intercept[SparkException] { - val result = KafkaRDD[String, String](sc, kafkaParams, badRanges, preferredHosts) + val result = KafkaUtils.createRDD[String, String](sc, kafkaParams, badRanges, preferredHosts) .map(_.value) .collect() } @@ -110,7 +111,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { kafkaTestUtils.sendMessages(topic, sent) var sentCount = sent.values.sum - val rdd = KafkaRDD[String, String](sc, kafkaParams, + val rdd = KafkaUtils.createRDD[String, String](sc, kafkaParams, Array(OffsetRange(topic, 0, 0, sentCount)), preferredHosts) val ranges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges @@ -121,7 +122,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { "didn't get all sent messages") // this is the "0 messages" case - val rdd2 = KafkaRDD[String, String](sc, kafkaParams, + val rdd2 = KafkaUtils.createRDD[String, String](sc, kafkaParams, Array(OffsetRange(topic, 0, sentCount, sentCount)), preferredHosts) // shouldn't get anything, since message is sent after rdd was defined @@ -132,7 +133,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { assert(rdd2.map(_.value).collect.size === 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 = KafkaRDD[String, String](sc, kafkaParams, + val rdd3 = KafkaUtils.createRDD[String, String](sc, kafkaParams, Array(OffsetRange(topic, 0, sentCount, sentCount + 1)), preferredHosts) // send lots of messages after rdd was defined, they shouldn't show up From 498585aa3433538256883a68aff474cc7311c9e9 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Sun, 26 Jun 2016 03:52:20 -0500 Subject: [PATCH 30/37] [SPARK-12177][Streaming][Kafka] make strategy for preferred location more explicit --- .../kafka/DirectKafkaInputDStream.scala | 15 +++-- .../spark/streaming/kafka/KafkaUtils.scala | 60 ++++++++----------- .../streaming/kafka/LocationStrategy.scala | 41 +++++++++++++ .../spark/streaming/kafka/OffsetRange.scala | 3 +- .../kafka/JavaDirectKafkaStreamSuite.java | 6 +- .../streaming/kafka/JavaKafkaRDDSuite.java | 5 +- .../kafka/DirectKafkaStreamSuite.scala | 2 +- .../spark/streaming/kafka/KafkaRDDSuite.scala | 2 +- 8 files changed, 80 insertions(+), 54 deletions(-) create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 47c6f7a494b0d..82e395fe6e365 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -44,9 +44,8 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number * of messages * per second that each '''partition''' will accept. - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[DirectKafkaInputDStream.preferConsistent]] - * Use [[DirectKafkaInputDStream.preferBrokers]] if your executors are on same nodes as brokers. + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. * @param executorKafkaParams Kafka * * configuration parameters. @@ -62,7 +61,7 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator @Experimental private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( _ssc: StreamingContext, - preferredHosts: ju.Map[TopicPartition, String], + locationStrategy: LocationStrategy, executorKafkaParams: ju.Map[String, Object], driverConsumer: () => Consumer[K, V] ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging with CanCommitOffsets { @@ -102,10 +101,10 @@ private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( } protected def getPreferredHosts: ju.Map[TopicPartition, String] = { - if (preferredHosts == KafkaUtils.preferBrokers) { - getBrokers - } else { - preferredHosts + locationStrategy match { + case PreferBrokers => getBrokers + case PreferConsistent => ju.Collections.emptyMap[TopicPartition, String]() + case PreferFixed(hostMap) => hostMap } } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 0761b87ae32a1..54ac4d7941942 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -45,9 +45,8 @@ object KafkaUtils extends Logging { * 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 - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[KafkaUtils.preferConsistent]] - * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -56,18 +55,21 @@ object KafkaUtils extends Logging { sc: SparkContext, kafkaParams: ju.Map[String, Object], offsetRanges: Array[OffsetRange], - preferredHosts: ju.Map[TopicPartition, String] + locationStrategy: LocationStrategy ): RDD[ConsumerRecord[K, V]] = { - assert(preferredHosts != KafkaUtils.preferBrokers, - "If you want to prefer brokers, you must provide a mapping for preferredHosts. " + - "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you. ") - + val preferredHosts = locationStrategy match { + case PreferBrokers => + throw new AssertionError( + "If you want to prefer brokers, you must provide a mapping using PreferFixed " + + "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you.") + case PreferConsistent => ju.Collections.emptyMap[TopicPartition, String]() + case PreferFixed(hostMap) => hostMap + } val kp = new ju.HashMap[String, Object](kafkaParams) fixKafkaParams(kp) val osr = offsetRanges.clone() - val ph = new ju.HashMap[TopicPartition, String](preferredHosts) - new KafkaRDD[K, V](sc, kp, osr, ph, true) + new KafkaRDD[K, V](sc, kp, osr, preferredHosts, true) } /** @@ -81,9 +83,8 @@ object KafkaUtils extends Logging { * 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 - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[KafkaUtils.preferConsistent]] - * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -94,28 +95,22 @@ object KafkaUtils extends Logging { valueClass: Class[V], kafkaParams: ju.Map[String, Object], offsetRanges: Array[OffsetRange], - preferredHosts: ju.Map[TopicPartition, String] + locationStrategy: LocationStrategy ): JavaRDD[ConsumerRecord[K, V]] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - new JavaRDD(createRDD[K, V](jsc.sc, kafkaParams, offsetRanges, preferredHosts)) + new JavaRDD(createRDD[K, V](jsc.sc, kafkaParams, offsetRanges, locationStrategy)) } - /** Prefer to run on kafka brokers, if they are on same hosts as executors */ - val preferBrokers: ju.Map[TopicPartition, String] = null - /** Prefer a consistent executor per TopicPartition, evenly from all executors */ - val preferConsistent: ju.Map[TopicPartition, String] = ju.Collections.emptyMap() - /** * Scala constructor for a DStream 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. - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[KafkaUtils.preferConsistent]] - * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. * @param executorKafkaParams Kafka * * configuration parameters. @@ -131,20 +126,15 @@ object KafkaUtils extends Logging { @Experimental def createDirectStream[K: ClassTag, V: ClassTag]( ssc: StreamingContext, - preferredHosts: ju.Map[TopicPartition, String], + locationStrategy: LocationStrategy, executorKafkaParams: ju.Map[String, Object], driverConsumer: () => Consumer[K, V] ): InputDStream[ConsumerRecord[K, V]] = { - val ph = if (preferredHosts == preferBrokers) { - preferredHosts - } else { - new ju.HashMap[TopicPartition, String](preferredHosts) - } val ekp = new ju.HashMap[String, Object](executorKafkaParams) fixKafkaParams(ekp) val cleaned = ssc.sparkContext.clean(driverConsumer) - new DirectKafkaInputDStream[K, V](ssc, ph, ekp, cleaned) + new DirectKafkaInputDStream[K, V](ssc, locationStrategy, ekp, cleaned) } /** @@ -152,10 +142,8 @@ object KafkaUtils extends Logging { * each given Kafka topic/partition corresponds to an RDD partition. * @param keyClass Class of the keys in the Kafka records * @param valueClass Class of the values in the Kafka records - * @param preferredHosts map from TopicPartition to preferred host for processing that partition. - * In most cases, use [[KafkaUtils.preferConsistent]] - * Use [[KafkaUtils.preferBrokers]] if your executors are on same nodes as brokers. - * @param executorKafkaParams Kafka + * @param locationStrategy In most cases, pass in [[PreferConsistent]], + * see [[LocationStrategy]] for more details. * * configuration parameters. * Requires "bootstrap.servers" to be set with Kafka broker(s), @@ -172,7 +160,7 @@ object KafkaUtils extends Logging { jssc: JavaStreamingContext, keyClass: Class[K], valueClass: Class[V], - preferredHosts: ju.Map[TopicPartition, String], + locationStrategy: LocationStrategy, executorKafkaParams: ju.Map[String, Object], driverConsumer: JFunction0[Consumer[K, V]] ): JavaInputDStream[ConsumerRecord[K, V]] = { @@ -182,7 +170,7 @@ object KafkaUtils extends Logging { new JavaInputDStream( createDirectStream[K, V]( - jssc.ssc, preferredHosts, executorKafkaParams, driverConsumer.call _)) + jssc.ssc, locationStrategy, executorKafkaParams, driverConsumer.call _)) } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala new file mode 100644 index 0000000000000..be99a723a2c8b --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala @@ -0,0 +1,41 @@ +package org.apache.spark.streaming.kafka + +import java.{ util => ju } + +import org.apache.kafka.common.TopicPartition + +import scala.collection.JavaConverters._ + +/** + * Choice of how to schedule consumers for a given TopicPartition on an executor. + * Kafka 0.10 consumers prefetch messages, so it's important for performance + * to keep cached consumers on appropriate executors, not recreate them for every partition. + * Choice of location is only a preference, not an absolute; partitions may be scheduled elsewhere. + */ +sealed trait LocationStrategy + +/** + * Use this only if your executors are on the same nodes as your Kafka brokers. + */ +case object PreferBrokers extends LocationStrategy { + def instance = this +} + +/** + * Use this in most cases, it will consistently distribute partitions across all executors. + */ +case object PreferConsistent extends LocationStrategy { + def instance = this +} + +/** + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ +case class PreferFixed(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy + +object PreferFixed { + def apply(hostMap: Map[TopicPartition, String]): PreferFixed = { + PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index 66ef2ce35d70d..11375b81d1d14 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -17,8 +17,9 @@ package org.apache.spark.streaming.kafka -import org.apache.kafka.common.TopicPartition import org.apache.kafka.clients.consumer.OffsetCommitCallback +import org.apache.kafka.common.TopicPartition + /** * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index bc4c8c344a8ff..fb05d0eeb758c 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -91,13 +91,11 @@ public void testKafkaStream() throws InterruptedException { kafkaParams.put("auto.offset.reset", "earliest"); kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); - Map preferredHosts = KafkaUtils.preferConsistent(); - JavaInputDStream istream1 = KafkaUtils.createDirectStream( ssc, String.class, String.class, - preferredHosts, + PreferConsistent.instance(), kafkaParams, new Function0>() { @Override @@ -139,7 +137,7 @@ public String call(ConsumerRecord r) { ssc, String.class, String.class, - preferredHosts, + PreferConsistent.instance(), kafkaParams2, new Function0>() { @Override diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 1e66054f54fb4..a20acf514cf4c 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -78,7 +78,6 @@ public void testKafkaRDD() throws InterruptedException { OffsetRange.create(topic2, 0, 0, 1) }; - Map emptyLeaders = new HashMap<>(); Map leaders = new HashMap<>(); String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); String broker = hostAndPort[0]; @@ -99,7 +98,7 @@ public String call(ConsumerRecord r) { String.class, kafkaParams, offsetRanges, - leaders + PreferFixed.apply(leaders) ).map(handler); JavaRDD rdd2 = KafkaUtils.createRDD( @@ -108,7 +107,7 @@ public String call(ConsumerRecord r) { String.class, kafkaParams, offsetRanges, - emptyLeaders + PreferConsistent.instance() ).map(handler); // just making sure the java user apis work; the scala tests handle logic corner cases diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index f016a8e6aa0dc..687d533604420 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -93,7 +93,7 @@ class DirectKafkaStreamSuite kp } - val preferredHosts = KafkaUtils.preferConsistent + val preferredHosts = PreferConsistent test("basic stream receiving with multiple topics and smallest starting offset") { val topics = List("basic1", "basic2", "basic3") diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index babd8868f66b6..46d6aa21287dd 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -59,7 +59,7 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { "group.id" -> s"test-consumer-${Random.nextInt}" ).asJava - private val preferredHosts = Map[TopicPartition, String]().asJava + private val preferredHosts = PreferConsistent test("basic usage") { val topic = s"topicbasic-${Random.nextInt}" From 83c9c5b517bffe2935a68df22c3734dcd4b9ce51 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Sun, 26 Jun 2016 13:38:25 -0500 Subject: [PATCH 31/37] [SPARK-12177][Streaming][Kafka] OOP style interface for configuring consumer --- .../streaming/kafka/ConsumerStrategy.scala | 164 ++++++++++++++++++ .../kafka/DirectKafkaInputDStream.scala | 16 +- .../spark/streaming/kafka/KafkaUtils.scala | 37 +--- .../streaming/kafka/LocationStrategy.scala | 26 ++- .../kafka/JavaDirectKafkaStreamSuite.java | 28 +-- .../streaming/kafka/JavaKafkaRDDSuite.java | 4 +- .../kafka/DirectKafkaStreamSuite.scala | 78 +++------ 7 files changed, 242 insertions(+), 111 deletions(-) create mode 100644 external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala new file mode 100644 index 0000000000000..e716e0fefd5fa --- /dev/null +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala @@ -0,0 +1,164 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka + +import java.{ util => ju } + +import scala.collection.JavaConverters._ +import scala.reflect.ClassTag + +import org.apache.kafka.clients.consumer._ +import org.apache.kafka.common.TopicPartition + +/** + * Choice of how to create and configure underlying Kafka Consumers on driver and executors. + * Kafka 0.10 consumers can require additional, sometimes complex, setup after object + * instantiation. This interface encapsulates that process, and allows it to be checkpointed. + * @tparam K type of Kafka message key + * @tparam V type of Kafka message value + */ +trait ConsumerStrategy[K, V] { + /** + * Kafka + * configuration parameters to be used on executors. Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + def executorKafkaParams: ju.Map[String, Object] + + /** + * Must return a fully configured Kafka Consumer, including subscribed or assigned topics. + * This consumer will be used on the driver to query for offsets only, not messages. + * @param currentOffsets A map from TopicPartition to offset, indicating how far the driver + * has successfully read. Will be empty on initial start, possibly non-empty on restart from + * checkpoint. + * TODO: is strategy or dstream responsible for seeking on checkpoint restart + */ + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] +} + +/** + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same parameters will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ +case class Subscribe[K: ClassTag, V: ClassTag]( + topics: ju.Collection[java.lang.String], + kafkaParams: ju.Map[String, Object] + ) extends ConsumerStrategy[K, V] { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.subscribe(topics) + consumer + } +} + +object Subscribe { + def create[K, V]( + keyClass: Class[K], + valueClass: Class[V], + topics: ju.Collection[java.lang.String], + kafkaParams: ju.Map[String, Object] + ): Subscribe[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + Subscribe[K, V](topics, kafkaParams) + } +} + +/** + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same parameters will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ +case class Assign[K: ClassTag, V: ClassTag]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object] + ) extends ConsumerStrategy[K, V] { + + def executorKafkaParams: ju.Map[String, Object] = kafkaParams + + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { + val consumer = new KafkaConsumer[K, V](kafkaParams) + consumer.assign(topicPartitions) + consumer + } +} + +object Assign { + def create[K, V]( + keyClass: Class[K], + valueClass: Class[V], + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object] + ): Assign[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + Assign[K, V](topicPartitions, kafkaParams) + } +} + + +/** + * Set offsets on initial startup only, after another strategy has configured consumer + * @param offsets: offsets to begin at + * @param init: ConsumerStrategy responsible for instantiation and initial config + */ +case class FromOffsets[K: ClassTag, V: ClassTag]( + offsets: Map[TopicPartition, Long], + init: ConsumerStrategy[K, V] +) extends ConsumerStrategy[K, V] { + def executorKafkaParams: ju.Map[String, Object] = init.executorKafkaParams + + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { + val consumer = init.onStart(currentOffsets) + + if (currentOffsets.isEmpty) { + offsets.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } + consumer + } +} + +object FromOffsets { + def create[K, V]( + keyClass: Class[K], + valueClass: Class[V], + offsets: ju.Map[TopicPartition, Long], + init: ConsumerStrategy[K, V] + ): FromOffsets[K, V] = { + implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) + implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + val off = Map(offsets.asScala.toSeq: _*) + FromOffsets[K, V](off, init) + } +} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 82e395fe6e365..9c239ce55db88 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -62,18 +62,24 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( _ssc: StreamingContext, locationStrategy: LocationStrategy, - executorKafkaParams: ju.Map[String, Object], - driverConsumer: () => Consumer[K, V] + consumerStrategy: ConsumerStrategy[K, V] ) extends InputDStream[ConsumerRecord[K, V]](_ssc) with Logging with CanCommitOffsets { + val executorKafkaParams = { + val ekp = new ju.HashMap[String, Object](consumerStrategy.executorKafkaParams) + KafkaUtils.fixKafkaParams(ekp) + ekp + } + + protected var currentOffsets = Map[TopicPartition, Long]() + @transient private var kc: Consumer[K, V] = null def consumer(): Consumer[K, V] = this.synchronized { if (null == kc) { - kc = driverConsumer() + kc = consumerStrategy.onStart(currentOffsets) } kc } - consumer() override def persist(newLevel: StorageLevel): DStream[ConsumerRecord[K, V]] = { logError("Kafka ConsumerRecord is not serializable. " + @@ -160,8 +166,6 @@ private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( } } - protected var currentOffsets = Map[TopicPartition, Long]() - protected def latestOffsets(): Map[TopicPartition, Long] = { val c = consumer c.poll(0) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 54ac4d7941942..8990a5fe66dd5 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -111,15 +111,8 @@ object KafkaUtils extends Logging { * per second that each '''partition''' will accept. * @param locationStrategy In most cases, pass in [[PreferConsistent]], * see [[LocationStrategy]] for more details. - * @param executorKafkaParams Kafka - * - * configuration parameters. - * Requires "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, - * and subscribe topics or assign partitions. - * This consumer will be used on the driver to query for offsets only, not messages. - * See Consumer doc + * @param consumerStrategy In most cases, pass in [[Subscribe]], + * see [[ConsumerStrategy]] for more details * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -127,14 +120,9 @@ object KafkaUtils extends Logging { def createDirectStream[K: ClassTag, V: ClassTag]( ssc: StreamingContext, locationStrategy: LocationStrategy, - executorKafkaParams: ju.Map[String, Object], - driverConsumer: () => Consumer[K, V] + consumerStrategy: ConsumerStrategy[K, V] ): InputDStream[ConsumerRecord[K, V]] = { - val ekp = new ju.HashMap[String, Object](executorKafkaParams) - fixKafkaParams(ekp) - val cleaned = ssc.sparkContext.clean(driverConsumer) - - new DirectKafkaInputDStream[K, V](ssc, locationStrategy, ekp, cleaned) + new DirectKafkaInputDStream[K, V](ssc, locationStrategy, consumerStrategy) } /** @@ -144,14 +132,8 @@ object KafkaUtils extends Logging { * @param valueClass Class of the values in the Kafka records * @param locationStrategy In most cases, pass in [[PreferConsistent]], * see [[LocationStrategy]] for more details. - * - * configuration parameters. - * Requires "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, - * and subscribe topics or assign partitions. - * This consumer will be used on the driver to query for offsets only, not messages. - * See Consumer doc + * @param consumerStrategy In most cases, pass in [[Subscribe]], + * see [[ConsumerStrategy]] for more details * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ @@ -161,19 +143,18 @@ object KafkaUtils extends Logging { keyClass: Class[K], valueClass: Class[V], locationStrategy: LocationStrategy, - executorKafkaParams: ju.Map[String, Object], - driverConsumer: JFunction0[Consumer[K, V]] + consumerStrategy: ConsumerStrategy[K, V] ): JavaInputDStream[ConsumerRecord[K, V]] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) + // TODO implement hasOffsetRanges etc so users dont have to rdd.rdd() new JavaInputDStream( createDirectStream[K, V]( - jssc.ssc, locationStrategy, executorKafkaParams, driverConsumer.call _)) + jssc.ssc, locationStrategy, consumerStrategy)) } - /** * Tweak kafka params to prevent issues on executors */ diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala index be99a723a2c8b..9092dcc43a917 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala @@ -1,10 +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. + */ + package org.apache.spark.streaming.kafka import java.{ util => ju } +import scala.collection.JavaConverters._ + import org.apache.kafka.common.TopicPartition -import scala.collection.JavaConverters._ /** * Choice of how to schedule consumers for a given TopicPartition on an executor. @@ -18,14 +36,14 @@ sealed trait LocationStrategy * Use this only if your executors are on the same nodes as your Kafka brokers. */ case object PreferBrokers extends LocationStrategy { - def instance = this + def create: PreferBrokers.type = this } /** * Use this in most cases, it will consistently distribute partitions across all executors. */ case object PreferConsistent extends LocationStrategy { - def instance = this + def create: PreferConsistent.type = this } /** @@ -38,4 +56,6 @@ object PreferFixed { def apply(hostMap: Map[TopicPartition, String]): PreferFixed = { PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) } + def create(hostMap: ju.Map[TopicPartition, String]): PreferFixed = + PreferFixed(hostMap) } diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index fb05d0eeb758c..ab8b7baba7488 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -91,20 +91,12 @@ public void testKafkaStream() throws InterruptedException { kafkaParams.put("auto.offset.reset", "earliest"); kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); - JavaInputDStream istream1 = KafkaUtils.createDirectStream( + JavaInputDStream> istream1 = KafkaUtils.createDirectStream( ssc, String.class, String.class, - PreferConsistent.instance(), - kafkaParams, - new Function0>() { - @Override - public Consumer call() { - KafkaConsumer consumer = new KafkaConsumer(kafkaParams); - consumer.subscribe(Arrays.asList(topic1)); - return consumer; - } - } + PreferConsistent.create(), + Subscribe.create(String.class, String.class, Arrays.asList(topic1), kafkaParams) ); JavaDStream stream1 = istream1.transform( @@ -133,20 +125,12 @@ public String call(ConsumerRecord r) { final Map kafkaParams2 = new HashMap<>(kafkaParams); kafkaParams2.put("group.id", "java-test-consumer-" + random.nextInt()); - JavaInputDStream istream2 = KafkaUtils.createDirectStream( + JavaInputDStream> istream2 = KafkaUtils.createDirectStream( ssc, String.class, String.class, - PreferConsistent.instance(), - kafkaParams2, - new Function0>() { - @Override - public Consumer call() { - KafkaConsumer consumer = new KafkaConsumer(kafkaParams2); - consumer.subscribe(Arrays.asList(topic2)); - return consumer; - } - } + PreferConsistent.create(), + Subscribe.create(String.class, String.class, Arrays.asList(topic2), kafkaParams2) ); JavaDStream stream2 = istream2.transform( diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index a20acf514cf4c..fa704edf67000 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -98,7 +98,7 @@ public String call(ConsumerRecord r) { String.class, kafkaParams, offsetRanges, - PreferFixed.apply(leaders) + PreferFixed.create(leaders) ).map(handler); JavaRDD rdd2 = KafkaUtils.createRDD( @@ -107,7 +107,7 @@ public String call(ConsumerRecord r) { String.class, kafkaParams, offsetRanges, - PreferConsistent.instance() + PreferConsistent.create() ).map(handler); // just making sure the java user apis work; the scala tests handle logic corner cases diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 687d533604420..be879e3200988 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -107,11 +107,8 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String](ssc, preferredHosts, kafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topics: _*)) - consumer - }) + KafkaUtils.createDirectStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topics: _*), kafkaParams)) } val allReceived = new ConcurrentLinkedQueue[(String, String)]() @@ -180,12 +177,8 @@ class DirectKafkaStreamSuite // Setup context and kafka stream with largest offset ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, - () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topic)) - consumer - }) + val s = new DirectKafkaInputDStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) s.consumer.poll(0) assert( s.consumer.position(topicPartition) >= offsetBeforeStart, @@ -232,13 +225,9 @@ class DirectKafkaStreamSuite // Setup context and kafka stream with largest offset ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { - val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, kafkaParams, - () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.assign(Arrays.asList(topicPartition)) - consumer.seek(topicPartition, 11L) - consumer - }) + val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, + FromOffsets[String, String](Map(topicPartition -> 11L), + Assign[String, String](Arrays.asList(topicPartition), kafkaParams))) s.consumer.poll(0) assert( s.consumer.position(topicPartition) >= offsetBeforeStart, @@ -275,11 +264,8 @@ class DirectKafkaStreamSuite // Setup the streaming context ssc = new StreamingContext(sparkConf, Milliseconds(100)) val kafkaStream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String](ssc, preferredHosts, kafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topic)) - consumer - }) + KafkaUtils.createDirectStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) } val keyedStream = kafkaStream.map { r => "key" -> r.value.toInt } val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => @@ -372,11 +358,7 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(100)) withClue("Error creating direct stream") { val kafkaStream = KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, kafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topic)) - consumer - }) + ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges val data = rdd.map(_.value).collect() @@ -427,11 +409,8 @@ class DirectKafkaStreamSuite ssc.addStreamingListener(collector) val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String](ssc, preferredHosts, kafkaParams, () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topic)) - consumer - }) + KafkaUtils.createDirectStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) } val allReceived = new ConcurrentLinkedQueue[(String, String)] @@ -504,12 +483,8 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) val kafkaStream = withClue("Error creating direct stream") { - new DirectKafkaInputDStream[String, String](ssc, preferredHosts, executorKafkaParams, - () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - consumer.subscribe(Arrays.asList(topic)) - consumer - }) { + new DirectKafkaInputDStream[String, String]( + ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) { override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) }.map(r => (r.key, r.value)) @@ -567,21 +542,24 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) val kafkaParams = getKafkaParams("auto.offset.reset" -> "earliest") - val executorKafkaParams = new JHashMap[String, Object](kafkaParams) - KafkaUtils.fixKafkaParams(executorKafkaParams) + val ekp = new JHashMap[String, Object](kafkaParams) + KafkaUtils.fixKafkaParams(ekp) val s = new DirectKafkaInputDStream[String, String]( ssc, preferredHosts, - executorKafkaParams, - () => { - val consumer = new KafkaConsumer[String, String](kafkaParams) - val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) - consumer.assign(Arrays.asList(tps: _*)) - tps.foreach(tp => consumer.seek(tp, 0)) - consumer - }) { - override protected[streaming] val rateController = mockRateController + new ConsumerStrategy[String, String] { + def executorKafkaParams = ekp + def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[String, String] = { + val consumer = new KafkaConsumer[String, String](kafkaParams) + val tps = List(new TopicPartition(topic, 0), new TopicPartition(topic, 1)) + consumer.assign(Arrays.asList(tps: _*)) + tps.foreach(tp => consumer.seek(tp, 0)) + consumer + } + } + ) { + override protected[streaming] val rateController = mockRateController } // manual start necessary because we arent consuming the stream, just checking its state s.start() From db9529066e9c9dab145f09f2332284f6869ed312 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 28 Jun 2016 00:52:41 -0500 Subject: [PATCH 32/37] [SPARK-12177][Streaming][Kafka] review feedback --- .../streaming/kafka/ConsumerStrategy.scala | 235 +++++++++++++----- .../kafka/DirectKafkaInputDStream.scala | 10 +- .../spark/streaming/kafka/KafkaRDD.scala | 5 +- .../spark/streaming/kafka/KafkaUtils.scala | 17 +- .../streaming/kafka/LocationStrategy.scala | 4 +- .../spark/streaming/kafka/package-info.java | 2 +- .../kafka/JavaConsumerStrategySuite.java | 84 +++++++ .../kafka/JavaDirectKafkaStreamSuite.java | 12 +- .../streaming/kafka/JavaKafkaRDDSuite.java | 8 +- .../kafka/JavaLocationStrategySuite.java | 58 +++++ .../kafka/DirectKafkaStreamSuite.scala | 18 +- 11 files changed, 347 insertions(+), 106 deletions(-) create mode 100644 external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaConsumerStrategySuite.java create mode 100644 external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaLocationStrategySuite.java diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala index e716e0fefd5fa..bebfa8dedcbc9 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.kafka import java.{ util => ju } import scala.collection.JavaConverters._ -import scala.reflect.ClassTag import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition @@ -46,7 +45,6 @@ trait ConsumerStrategy[K, V] { * @param currentOffsets A map from TopicPartition to offset, indicating how far the driver * has successfully read. Will be empty on initial start, possibly non-empty on restart from * checkpoint. - * TODO: is strategy or dstream responsible for seeking on checkpoint restart */ def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] } @@ -56,14 +54,18 @@ trait ConsumerStrategy[K, V] { * @param topics collection of topics to subscribe * @param kafkaParams Kafka * - * configuration parameters to be used on driver. The same parameters will be used on executors, + * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. */ -case class Subscribe[K: ClassTag, V: ClassTag]( +case class Subscribe[K, V] private( topics: ju.Collection[java.lang.String], - kafkaParams: ju.Map[String, Object] + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long] ) extends ConsumerStrategy[K, V] { def executorKafkaParams: ju.Map[String, Object] = kafkaParams @@ -71,36 +73,117 @@ case class Subscribe[K: ClassTag, V: ClassTag]( def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { val consumer = new KafkaConsumer[K, V](kafkaParams) consumer.subscribe(topics) + if (currentOffsets.isEmpty) { + offsets.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } + consumer } } +/** + * Companion object for creating [[Subscribe]] strategy + */ object Subscribe { + /** + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + def apply[K, V]( + topics: Iterable[java.lang.String], + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): Subscribe[K, V] = { + Subscribe[K, V]( + new ju.ArrayList(topics.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, Long](offsets.asJava)) + } + + /** + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + def apply[K, V]( + topics: Iterable[java.lang.String], + kafkaParams: collection.Map[String, Object]): Subscribe[K, V] = { + Subscribe[K, V]( + new ju.ArrayList(topics.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, Long]()) + } + + /** + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + def create[K, V]( + topics: ju.Collection[java.lang.String], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long]): Subscribe[K, V] = { + Subscribe[K, V](topics, kafkaParams, offsets) + } + + /** + * Subscribe to a collection of topics. + * @param topics collection of topics to subscribe + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ def create[K, V]( - keyClass: Class[K], - valueClass: Class[V], topics: ju.Collection[java.lang.String], - kafkaParams: ju.Map[String, Object] - ): Subscribe[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - Subscribe[K, V](topics, kafkaParams) + kafkaParams: ju.Map[String, Object]): Subscribe[K, V] = { + Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, Long]()) } + } /** * Assign a fixed collection of TopicPartitions - * @param topicPartitions collection of TopicPartitions to subscribe + * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka * - * configuration parameters to be used on driver. The same parameters will be used on executors, + * configuration parameters to be used on driver. The same params will be used on executors, * with minor automatic modifications applied. * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. */ -case class Assign[K: ClassTag, V: ClassTag]( +case class Assign[K, V] private( topicPartitions: ju.Collection[TopicPartition], - kafkaParams: ju.Map[String, Object] + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long] ) extends ConsumerStrategy[K, V] { def executorKafkaParams: ju.Map[String, Object] = kafkaParams @@ -108,57 +191,95 @@ case class Assign[K: ClassTag, V: ClassTag]( def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { val consumer = new KafkaConsumer[K, V](kafkaParams) consumer.assign(topicPartitions) - consumer - } -} + if (currentOffsets.isEmpty) { + offsets.asScala.foreach { case (topicPartition, offset) => + consumer.seek(topicPartition, offset) + } + } -object Assign { - def create[K, V]( - keyClass: Class[K], - valueClass: Class[V], - topicPartitions: ju.Collection[TopicPartition], - kafkaParams: ju.Map[String, Object] - ): Assign[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - Assign[K, V](topicPartitions, kafkaParams) + consumer } } - /** - * Set offsets on initial startup only, after another strategy has configured consumer - * @param offsets: offsets to begin at - * @param init: ConsumerStrategy responsible for instantiation and initial config + * Companion object for creating [[Assign]] strategy */ -case class FromOffsets[K: ClassTag, V: ClassTag]( - offsets: Map[TopicPartition, Long], - init: ConsumerStrategy[K, V] -) extends ConsumerStrategy[K, V] { - def executorKafkaParams: ju.Map[String, Object] = init.executorKafkaParams +object Assign { + /** + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + def apply[K, V]( + topicPartitions: Iterable[TopicPartition], + kafkaParams: collection.Map[String, Object], + offsets: collection.Map[TopicPartition, Long]): Assign[K, V] = { + Assign[K, V]( + new ju.ArrayList(topicPartitions.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + new ju.HashMap[TopicPartition, Long](offsets.asJava)) + } - def onStart(currentOffsets: Map[TopicPartition, Long]): Consumer[K, V] = { - val consumer = init.onStart(currentOffsets) + /** + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ + def apply[K, V]( + topicPartitions: Iterable[TopicPartition], + kafkaParams: collection.Map[String, Object]): Assign[K, V] = { + Assign[K, V]( + new ju.ArrayList(topicPartitions.asJavaCollection), + new ju.HashMap[String, Object](kafkaParams.asJava), + ju.Collections.emptyMap[TopicPartition, Long]()) + } - if (currentOffsets.isEmpty) { - offsets.foreach { case (topicPartition, offset) => - consumer.seek(topicPartition, offset) - } - } - consumer + /** + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + * @param offsets: offsets to begin at on initial startup. If no offset is given for a + * TopicPartition, the committed offset (if applicable) or kafka param + * auto.offset.reset will be used. + */ + def create[K, V]( + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object], + offsets: ju.Map[TopicPartition, Long]): Assign[K, V] = { + Assign[K, V](topicPartitions, kafkaParams, offsets) } -} -object FromOffsets { + /** + * Assign a fixed collection of TopicPartitions + * @param topicPartitions collection of TopicPartitions to assign + * @param kafkaParams Kafka + * + * configuration parameters to be used on driver. The same params will be used on executors, + * with minor automatic modifications applied. + * Requires "bootstrap.servers" to be set + * with Kafka broker(s) specified in host1:port1,host2:port2 form. + */ def create[K, V]( - keyClass: Class[K], - valueClass: Class[V], - offsets: ju.Map[TopicPartition, Long], - init: ConsumerStrategy[K, V] - ): FromOffsets[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - val off = Map(offsets.asScala.toSeq: _*) - FromOffsets[K, V](off, init) + topicPartitions: ju.Collection[TopicPartition], + kafkaParams: ju.Map[String, Object]): Assign[K, V] = { + Assign[K, V](topicPartitions, kafkaParams, ju.Collections.emptyMap[TopicPartition, Long]()) } } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 9c239ce55db88..dcd1c6f73e4b8 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -24,7 +24,6 @@ import java.util.concurrent.atomic.AtomicReference import scala.annotation.tailrec import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.reflect.ClassTag import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.{ PartitionInfo, TopicPartition } @@ -59,7 +58,7 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator * @tparam V type of Kafka message value */ @Experimental -private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( +private[spark] class DirectKafkaInputDStream[K, V]( _ssc: StreamingContext, locationStrategy: LocationStrategy, consumerStrategy: ConsumerStrategy[K, V] @@ -166,6 +165,9 @@ private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( } } + /** + * Returns the latest (highest) available offsets, taking new partitions into account. + */ protected def latestOffsets(): Map[TopicPartition, Long] = { val c = consumer c.poll(0) @@ -173,9 +175,11 @@ private[spark] class DirectKafkaInputDStream[K: ClassTag, V: ClassTag]( // make sure new partitions are reflected in currentOffsets val newPartitions = parts.diff(currentOffsets.keySet) + // position for new partitions determined by auto.offset.reset if no commit currentOffsets = currentOffsets ++ newPartitions.map(tp => tp -> c.position(tp)).toMap + // don't want to consume messages, so pause c.pause(newPartitions.asJava) - + // find latest available offsets c.seekToEnd(currentOffsets.keySet.asJava) parts.map(tp => tp -> c.position(tp)).toMap } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 4c87a3af74881..697c6bf46e78c 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.kafka import java.{ util => ju } import scala.collection.mutable.ArrayBuffer -import scala.reflect.{classTag, ClassTag} import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } import org.apache.kafka.common.TopicPartition @@ -50,9 +49,7 @@ import org.apache.spark.storage.StorageLevel * @tparam V type of Kafka message value */ @Experimental -private[spark] class KafkaRDD[ - K: ClassTag, - V: ClassTag]( +private[spark] class KafkaRDD[K, V]( sc: SparkContext, val kafkaParams: ju.Map[String, Object], val offsetRanges: Array[OffsetRange], diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 8990a5fe66dd5..62badff27573f 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -19,8 +19,6 @@ package org.apache.spark.streaming.kafka import java.{ util => ju } -import scala.reflect.{classTag, ClassTag} - import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition @@ -51,7 +49,7 @@ object KafkaUtils extends Logging { * @tparam V type of Kafka message value */ @Experimental - def createRDD[K: ClassTag, V: ClassTag]( + def createRDD[K, V]( sc: SparkContext, kafkaParams: ju.Map[String, Object], offsetRanges: Array[OffsetRange], @@ -91,14 +89,10 @@ object KafkaUtils extends Logging { @Experimental def createRDD[K, V]( jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], kafkaParams: ju.Map[String, Object], offsetRanges: Array[OffsetRange], locationStrategy: LocationStrategy ): JavaRDD[ConsumerRecord[K, V]] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) new JavaRDD(createRDD[K, V](jsc.sc, kafkaParams, offsetRanges, locationStrategy)) } @@ -117,7 +111,7 @@ object KafkaUtils extends Logging { * @tparam V type of Kafka message value */ @Experimental - def createDirectStream[K: ClassTag, V: ClassTag]( + def createDirectStream[K, V]( ssc: StreamingContext, locationStrategy: LocationStrategy, consumerStrategy: ConsumerStrategy[K, V] @@ -140,16 +134,9 @@ object KafkaUtils extends Logging { @Experimental def createDirectStream[K, V]( jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], locationStrategy: LocationStrategy, consumerStrategy: ConsumerStrategy[K, V] ): JavaInputDStream[ConsumerRecord[K, V]] = { - - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - - // TODO implement hasOffsetRanges etc so users dont have to rdd.rdd() new JavaInputDStream( createDirectStream[K, V]( jssc.ssc, locationStrategy, consumerStrategy)) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala index 9092dcc43a917..7ab7ac41a09a2 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala @@ -50,10 +50,10 @@ case object PreferConsistent extends LocationStrategy { * Use this to place particular TopicPartitions on particular hosts if your load is uneven. * Any TopicPartition not specified in the map will use a consistent location. */ -case class PreferFixed(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy +case class PreferFixed private(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy object PreferFixed { - def apply(hostMap: Map[TopicPartition, String]): PreferFixed = { + def apply(hostMap: collection.Map[TopicPartition, String]): PreferFixed = { PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) } def create(hostMap: ju.Map[TopicPartition, String]): PreferFixed = diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java index 947bae115a620..2e5ab0fb3bef9 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java @@ -18,4 +18,4 @@ /** * Kafka receiver for spark streaming. */ -package org.apache.spark.streaming.kafka; \ No newline at end of file +package org.apache.spark.streaming.kafka; diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaConsumerStrategySuite.java new file mode 100644 index 0000000000000..2c97b76566b28 --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaConsumerStrategySuite.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.*; + +import scala.collection.JavaConverters; + +import org.apache.kafka.common.TopicPartition; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaConsumerStrategySuite implements Serializable { + + @Test + public void testConsumerStrategyConstructors() { + final String topic1 = "topic1"; + final Collection topics = Arrays.asList(topic1); + final scala.collection.Iterable sTopics = + JavaConverters.collectionAsScalaIterableConverter(topics).asScala(); + final TopicPartition tp1 = new TopicPartition(topic1, 0); + final TopicPartition tp2 = new TopicPartition(topic1, 1); + final Collection parts = Arrays.asList(tp1, tp2); + final scala.collection.Iterable sParts = + JavaConverters.collectionAsScalaIterableConverter(parts).asScala(); + final Map kafkaParams = new HashMap(); + kafkaParams.put("bootstrap.servers", "not used"); + final scala.collection.Map sKafkaParams = + JavaConverters.mapAsScalaMapConverter(kafkaParams).asScala(); + final Map offsets = new HashMap<>(); + offsets.put(tp1, 23L); + final scala.collection.Map sOffsets = + JavaConverters.mapAsScalaMapConverter(offsets).asScala(); + + // make sure constructors can be called from java + final ConsumerStrategy sub0 = + Subscribe.apply(topics, kafkaParams, offsets); + final ConsumerStrategy sub1 = + Subscribe.apply(sTopics, sKafkaParams, sOffsets); + final ConsumerStrategy sub2 = + Subscribe.apply(sTopics, sKafkaParams); + final ConsumerStrategy sub3 = + Subscribe.create(topics, kafkaParams, offsets); + final ConsumerStrategy sub4 = + Subscribe.create(topics, kafkaParams); + + Assert.assertEquals( + sub1.executorKafkaParams().get("bootstrap.servers"), + sub3.executorKafkaParams().get("bootstrap.servers")); + + final ConsumerStrategy asn0 = + Assign.apply(parts, kafkaParams, offsets); + final ConsumerStrategy asn1 = + Assign.apply(sParts, sKafkaParams, sOffsets); + final ConsumerStrategy asn2 = + Assign.apply(sParts, sKafkaParams); + final ConsumerStrategy asn3 = + Assign.create(parts, kafkaParams, offsets); + final ConsumerStrategy asn4 = + Assign.create(parts, kafkaParams); + + Assert.assertEquals( + asn1.executorKafkaParams().get("bootstrap.servers"), + asn3.executorKafkaParams().get("bootstrap.servers")); + } + +} diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index ab8b7baba7488..03f0404ace6c9 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -22,10 +22,7 @@ import java.util.concurrent.atomic.AtomicReference; import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.TopicPartition; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.clients.consumer.Consumer; -import org.apache.kafka.clients.consumer.KafkaConsumer; import org.junit.After; import org.junit.Assert; @@ -35,7 +32,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.Function0; import org.apache.spark.api.java.function.VoidFunction; import org.apache.spark.streaming.Durations; import org.apache.spark.streaming.api.java.JavaDStream; @@ -93,10 +89,8 @@ public void testKafkaStream() throws InterruptedException { JavaInputDStream> istream1 = KafkaUtils.createDirectStream( ssc, - String.class, - String.class, PreferConsistent.create(), - Subscribe.create(String.class, String.class, Arrays.asList(topic1), kafkaParams) + Subscribe.create(Arrays.asList(topic1), kafkaParams) ); JavaDStream stream1 = istream1.transform( @@ -127,10 +121,8 @@ public String call(ConsumerRecord r) { JavaInputDStream> istream2 = KafkaUtils.createDirectStream( ssc, - String.class, - String.class, PreferConsistent.create(), - Subscribe.create(String.class, String.class, Arrays.asList(topic2), kafkaParams2) + Subscribe.create(Arrays.asList(topic2), kafkaParams2) ); JavaDStream stream2 = istream2.transform( diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index fa704edf67000..15a445379b1a8 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -92,19 +92,15 @@ public String call(ConsumerRecord r) { } }; - JavaRDD rdd1 = KafkaUtils.createRDD( + JavaRDD rdd1 = KafkaUtils.createRDD( sc, - String.class, - String.class, kafkaParams, offsetRanges, PreferFixed.create(leaders) ).map(handler); - JavaRDD rdd2 = KafkaUtils.createRDD( + JavaRDD rdd2 = KafkaUtils.createRDD( sc, - String.class, - String.class, kafkaParams, offsetRanges, PreferConsistent.create() diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaLocationStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaLocationStrategySuite.java new file mode 100644 index 0000000000000..e638b6c5d960e --- /dev/null +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaLocationStrategySuite.java @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.kafka; + +import java.io.Serializable; +import java.util.*; + +import scala.collection.JavaConverters; + +import org.apache.kafka.common.TopicPartition; + +import org.junit.Assert; +import org.junit.Test; + +public class JavaLocationStrategySuite implements Serializable { + + @Test + public void testLocationStrategyConstructors() { + final String topic1 = "topic1"; + final TopicPartition tp1 = new TopicPartition(topic1, 0); + final TopicPartition tp2 = new TopicPartition(topic1, 1); + final Map hosts = new HashMap<>(); + hosts.put(tp1, "node1"); + hosts.put(tp2, "node2"); + final scala.collection.Map sHosts = + JavaConverters.mapAsScalaMapConverter(hosts).asScala(); + + // make sure constructors can be called from java + final LocationStrategy c1 = PreferConsistent.create(); + final LocationStrategy c2 = PreferConsistent$.MODULE$; + Assert.assertEquals(c1, c2); + + final LocationStrategy c3 = PreferBrokers.create(); + final LocationStrategy c4 = PreferBrokers$.MODULE$; + Assert.assertEquals(c3, c4); + + final LocationStrategy c5 = PreferFixed.create(hosts); + final LocationStrategy c6 = PreferFixed.apply(sHosts); + Assert.assertEquals(c5, c6); + + } + +} diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index be879e3200988..5032f4c9831b8 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -108,7 +108,7 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topics: _*), kafkaParams)) + ssc, preferredHosts, Subscribe[String, String](topics, kafkaParams.asScala)) } val allReceived = new ConcurrentLinkedQueue[(String, String)]() @@ -178,7 +178,7 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { val s = new DirectKafkaInputDStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) s.consumer.poll(0) assert( s.consumer.position(topicPartition) >= offsetBeforeStart, @@ -226,8 +226,10 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(200)) val stream = withClue("Error creating direct stream") { val s = new DirectKafkaInputDStream[String, String](ssc, preferredHosts, - FromOffsets[String, String](Map(topicPartition -> 11L), - Assign[String, String](Arrays.asList(topicPartition), kafkaParams))) + Assign[String, String]( + List(topicPartition), + kafkaParams.asScala, + Map(topicPartition -> 11L))) s.consumer.poll(0) assert( s.consumer.position(topicPartition) >= offsetBeforeStart, @@ -265,7 +267,7 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(100)) val kafkaStream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) } val keyedStream = kafkaStream.map { r => "key" -> r.value.toInt } val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => @@ -358,7 +360,7 @@ class DirectKafkaStreamSuite ssc = new StreamingContext(sparkConf, Milliseconds(100)) withClue("Error creating direct stream") { val kafkaStream = KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) kafkaStream.foreachRDD { (rdd: RDD[ConsumerRecord[String, String]], time: Time) => val offsets = rdd.asInstanceOf[HasOffsetRanges].offsetRanges val data = rdd.map(_.value).collect() @@ -410,7 +412,7 @@ class DirectKafkaStreamSuite val stream = withClue("Error creating direct stream") { KafkaUtils.createDirectStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) } val allReceived = new ConcurrentLinkedQueue[(String, String)] @@ -484,7 +486,7 @@ class DirectKafkaStreamSuite val kafkaStream = withClue("Error creating direct stream") { new DirectKafkaInputDStream[String, String]( - ssc, preferredHosts, Subscribe[String, String](Arrays.asList(topic), kafkaParams)) { + ssc, preferredHosts, Subscribe[String, String](List(topic), kafkaParams.asScala)) { override protected[streaming] val rateController = Some(new DirectKafkaRateController(id, estimator)) }.map(r => (r.key, r.value)) From b1eec577d64f82784afaf626ad5a325bc7a1d555 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Tue, 28 Jun 2016 21:35:20 -0500 Subject: [PATCH 33/37] [SPARK-12177][Streaming][Kafka] review feedback --- .../streaming/kafka/CachedKafkaConsumer.scala | 45 ++++++++++--------- .../kafka/DirectKafkaInputDStream.scala | 1 - .../spark/streaming/kafka/KafkaRDD.scala | 1 - .../spark/streaming/kafka/package-info.java | 2 +- .../spark/streaming/kafka/package.scala | 2 +- .../kafka/JavaDirectKafkaStreamSuite.java | 11 ++--- .../kafka/DirectKafkaStreamSuite.scala | 2 +- .../spark/streaming/kafka/KafkaRDDSuite.scala | 6 +-- 8 files changed, 35 insertions(+), 35 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala index 5837d6f1e930f..b48a28297c2f4 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala @@ -20,7 +20,7 @@ package org.apache.spark.streaming.kafka import java.{ util => ju } import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord, KafkaConsumer } -import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.{ KafkaException, TopicPartition } import org.apache.spark.SparkConf import org.apache.spark.internal.Logging @@ -63,9 +63,9 @@ class CachedKafkaConsumer[K, V] private( * Sequential forward access will use buffers, but random access will be horribly inefficient. */ def get(offset: Long, timeout: Long): ConsumerRecord[K, V] = { - log.debug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") + logDebug(s"Get $groupId $topic $partition nextOffset $nextOffset requested $offset") if (offset != nextOffset) { - log.info(s"Initial fetch for $groupId $topic $partition $offset") + logInfo(s"Initial fetch for $groupId $topic $partition $offset") seek(offset) poll(timeout) } @@ -76,7 +76,7 @@ class CachedKafkaConsumer[K, V] private( var record = buffer.next() if (record.offset != offset) { - log.info(s"Buffer miss for $groupId $topic $partition $offset") + logInfo(s"Buffer miss for $groupId $topic $partition $offset") seek(offset) poll(timeout) assert(buffer.hasNext(), @@ -91,14 +91,14 @@ class CachedKafkaConsumer[K, V] private( } private def seek(offset: Long): Unit = { - log.debug(s"Seeking to $topicPartition $offset") + logDebug(s"Seeking to $topicPartition $offset") consumer.seek(topicPartition, offset) } private def poll(timeout: Long): Unit = { val p = consumer.poll(timeout) val r = p.records(topicPartition) - log.debug(s"Polled ${p.partitions()} ${r.size}") + logDebug(s"Polled ${p.partitions()} ${r.size}") buffer = r.iterator } @@ -118,13 +118,18 @@ object CachedKafkaConsumer extends Logging { maxCapacity: Int, loadFactor: Float): Unit = CachedKafkaConsumer.synchronized { if (null == cache) { - log.info(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") + logInfo(s"Initializing cache $initialCapacity $maxCapacity $loadFactor") cache = new ju.LinkedHashMap[CacheKey, CachedKafkaConsumer[_, _]]( initialCapacity, loadFactor, true) { override def removeEldestEntry( entry: ju.Map.Entry[CacheKey, CachedKafkaConsumer[_, _]]): Boolean = { if (this.size > maxCapacity) { - entry.getValue.consumer.close() + try { + entry.getValue.consumer.close() + } catch { + case x: KafkaException => + logError("Error closing oldest Kafka consumer", x) + } true } else { false @@ -147,8 +152,8 @@ object CachedKafkaConsumer extends Logging { val k = CacheKey(groupId, topic, partition) val v = cache.get(k) if (null == v) { - log.info(s"Cache miss for $k") - log.debug(cache.keySet.toString) + logInfo(s"Cache miss for $k") + logDebug(cache.keySet.toString) val c = new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) cache.put(k, c) c @@ -170,15 +175,15 @@ object CachedKafkaConsumer extends Logging { new CachedKafkaConsumer[K, V](groupId, topic, partition, kafkaParams) /** remove consumer for given groupId, topic, and partition, if it exists */ - def remove(groupId: String, topic: String, partition: Int): Unit = - CachedKafkaConsumer.synchronized { - val k = CacheKey(groupId, topic, partition) - log.info(s"Removing $k from cache") - val v = cache.get(k) - if (null != v) { - v.close() - cache.remove(k) - log.info(s"Removed $k from cache") - } + def remove(groupId: String, topic: String, partition: Int): Unit = { + val k = CacheKey(groupId, topic, partition) + logInfo(s"Removing $k from cache") + val v = CachedKafkaConsumer.synchronized { + cache.remove(k) + } + if (null != v) { + v.close() + logInfo(s"Removed $k from cache") } + } } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index dcd1c6f73e4b8..4274bc3c1226b 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -57,7 +57,6 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ -@Experimental private[spark] class DirectKafkaInputDStream[K, V]( _ssc: StreamingContext, locationStrategy: LocationStrategy, diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 697c6bf46e78c..9a32c89ec5014 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -48,7 +48,6 @@ import org.apache.spark.storage.StorageLevel * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ -@Experimental private[spark] class KafkaRDD[K, V]( sc: SparkContext, val kafkaParams: ju.Map[String, Object], diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java index 2e5ab0fb3bef9..f264c322fa50d 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java @@ -16,6 +16,6 @@ */ /** - * Kafka receiver for spark streaming. + * Spark Integration for Kafka */ package org.apache.spark.streaming.kafka; diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala index 47c5187f8751f..4dfd41d602e8d 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala @@ -18,6 +18,6 @@ package org.apache.spark.streaming /** - * Kafka receiver for spark streaming, + * Spark Integration for Kafka */ package object kafka diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index 03f0404ace6c9..1fc1cdb52d3e3 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -85,7 +85,8 @@ public void testKafkaStream() throws InterruptedException { kafkaParams.put("key.deserializer", StringDeserializer.class); kafkaParams.put("value.deserializer", StringDeserializer.class); kafkaParams.put("auto.offset.reset", "earliest"); - kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt()); + kafkaParams.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); JavaInputDStream> istream1 = KafkaUtils.createDirectStream( ssc, @@ -117,7 +118,8 @@ public String call(ConsumerRecord r) { ); final Map kafkaParams2 = new HashMap<>(kafkaParams); - kafkaParams2.put("group.id", "java-test-consumer-" + random.nextInt()); + kafkaParams2.put("group.id", "java-test-consumer-" + random.nextInt() + + "-" + System.currentTimeMillis()); JavaInputDStream> istream2 = KafkaUtils.createDirectStream( ssc, @@ -155,11 +157,6 @@ public String call(ConsumerRecord r) { @Override 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() - ); - } } } ); diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 5032f4c9831b8..cfb0f41fb3031 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -88,7 +88,7 @@ class DirectKafkaStreamSuite kp.put("bootstrap.servers", kafkaTestUtils.brokerAddress) kp.put("key.deserializer", classOf[StringDeserializer]) kp.put("value.deserializer", classOf[StringDeserializer]) - kp.put("group.id", s"test-consumer-${Random.nextInt}") + kp.put("group.id", s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") extra.foreach(e => kp.put(e._1, e._2)) kp } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 46d6aa21287dd..9bfbe8d8f2f11 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -56,13 +56,13 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { "bootstrap.servers" -> kafkaTestUtils.brokerAddress, "key.deserializer" -> classOf[StringDeserializer], "value.deserializer" -> classOf[StringDeserializer], - "group.id" -> s"test-consumer-${Random.nextInt}" + "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}" ).asJava private val preferredHosts = PreferConsistent test("basic usage") { - val topic = s"topicbasic-${Random.nextInt}" + val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" kafkaTestUtils.createTopic(topic) val messages = Array("the", "quick", "brown", "fox") kafkaTestUtils.sendMessages(topic, messages) @@ -101,7 +101,7 @@ 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"topicboundary-${Random.nextInt}" + val topic = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) kafkaTestUtils.createTopic(topic) From 31502d9416207f40a5ffed54fb694669758dbf8d Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 29 Jun 2016 15:01:47 -0500 Subject: [PATCH 34/37] [SPARK-12177][Streaming][Kafka] review feedback --- .../spark/streaming/kafka/KafkaRDD.scala | 11 ++++++++- .../spark/streaming/kafka/KafkaRDDSuite.scala | 23 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 9a32c89ec5014..1ad9963039b9e 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -133,9 +133,18 @@ private[spark] class KafkaRDD[K, V]( val bm = sparkContext.env.blockManager bm.master.getPeers(bm.blockManagerId).toArray .map(x => ExecutorCacheTaskLocation(x.host, x.executorId)) - .sortWith((a, b) => a.host > b.host || a.executorId > b.executorId) + .sortWith(compareExecutors) } + protected[kafka] def compareExecutors( + a: ExecutorCacheTaskLocation, + b: ExecutorCacheTaskLocation): Boolean = + if (a.host == b.host) { + a.executorId > b.executorId + } else { + a.host > b.host + } + /** * Non-negative modulus, from java 8 math */ diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index 9bfbe8d8f2f11..7aa9dc4a2bc8b 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.streaming.kafka +import java.{ util => ju } + import scala.collection.JavaConverters._ import scala.util.Random @@ -25,6 +27,7 @@ import org.apache.kafka.common.serialization.StringDeserializer import org.scalatest.BeforeAndAfterAll import org.apache.spark._ +import org.apache.spark.scheduler.ExecutorCacheTaskLocation class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { @@ -143,4 +146,24 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { "didn't get exactly one message") } + test("executor sorting") { + val kafkaParams = new ju.HashMap[String, Object](getKafkaParams()) + kafkaParams.put("auto.offset.reset", "none") + val rdd = new KafkaRDD[String, String]( + sc, + kafkaParams, + Array(OffsetRange("unused", 0, 1, 2)), + ju.Collections.emptyMap[TopicPartition, String](), + true) + val a3 = ExecutorCacheTaskLocation("a", "3") + val a4 = ExecutorCacheTaskLocation("a", "4") + val b1 = ExecutorCacheTaskLocation("b", "1") + val b2 = ExecutorCacheTaskLocation("b", "2") + + val correct = Array(b2, b1, a4, a3) + + correct.permutations.foreach { p => + assert(p.sortWith(rdd.compareExecutors) === correct) + } + } } From f86336951d4dd196812420e4e902f105ea95e81b Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 29 Jun 2016 17:09:11 -0500 Subject: [PATCH 35/37] [SPARK-12177][Streaming][Kafka] review feedback --- .../streaming/kafka/ConsumerStrategy.scala | 29 +++++++++++++++++++ .../kafka/DirectKafkaInputDStream.scala | 1 - .../spark/streaming/kafka/KafkaRDD.scala | 1 - .../spark/streaming/kafka/KafkaUtils.scala | 8 +++++ .../streaming/kafka/LocationStrategy.scala | 16 ++++++++++ .../spark/streaming/kafka/OffsetRange.scala | 9 ++++++ 6 files changed, 62 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala index bebfa8dedcbc9..23016bf664f6b 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala @@ -24,13 +24,18 @@ import scala.collection.JavaConverters._ import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition +import org.apache.spark.annotation.Experimental + + /** + * :: Experimental :: * Choice of how to create and configure underlying Kafka Consumers on driver and executors. * Kafka 0.10 consumers can require additional, sometimes complex, setup after object * instantiation. This interface encapsulates that process, and allows it to be checkpointed. * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ +@Experimental trait ConsumerStrategy[K, V] { /** * Kafka @@ -50,6 +55,7 @@ trait ConsumerStrategy[K, V] { } /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -62,6 +68,7 @@ trait ConsumerStrategy[K, V] { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ +@Experimental case class Subscribe[K, V] private( topics: ju.Collection[java.lang.String], kafkaParams: ju.Map[String, Object], @@ -84,10 +91,13 @@ case class Subscribe[K, V] private( } /** + * :: Experimental :: * Companion object for creating [[Subscribe]] strategy */ +@Experimental object Subscribe { /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -100,6 +110,7 @@ object Subscribe { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def apply[K, V]( topics: Iterable[java.lang.String], kafkaParams: collection.Map[String, Object], @@ -111,6 +122,7 @@ object Subscribe { } /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -120,6 +132,7 @@ object Subscribe { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def apply[K, V]( topics: Iterable[java.lang.String], kafkaParams: collection.Map[String, Object]): Subscribe[K, V] = { @@ -130,6 +143,7 @@ object Subscribe { } /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -142,6 +156,7 @@ object Subscribe { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def create[K, V]( topics: ju.Collection[java.lang.String], kafkaParams: ju.Map[String, Object], @@ -150,6 +165,7 @@ object Subscribe { } /** + * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -159,6 +175,7 @@ object Subscribe { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def create[K, V]( topics: ju.Collection[java.lang.String], kafkaParams: ju.Map[String, Object]): Subscribe[K, V] = { @@ -168,6 +185,7 @@ object Subscribe { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -180,6 +198,7 @@ object Subscribe { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ +@Experimental case class Assign[K, V] private( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object], @@ -202,10 +221,13 @@ case class Assign[K, V] private( } /** + * :: Experimental :: * Companion object for creating [[Assign]] strategy */ +@Experimental object Assign { /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -218,6 +240,7 @@ object Assign { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def apply[K, V]( topicPartitions: Iterable[TopicPartition], kafkaParams: collection.Map[String, Object], @@ -229,6 +252,7 @@ object Assign { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -238,6 +262,7 @@ object Assign { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def apply[K, V]( topicPartitions: Iterable[TopicPartition], kafkaParams: collection.Map[String, Object]): Assign[K, V] = { @@ -248,6 +273,7 @@ object Assign { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -260,6 +286,7 @@ object Assign { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ + @Experimental def create[K, V]( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object], @@ -268,6 +295,7 @@ object Assign { } /** + * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -277,6 +305,7 @@ object Assign { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ + @Experimental def create[K, V]( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object]): Assign[K, V] = { diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 4274bc3c1226b..324824b56cea2 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -28,7 +28,6 @@ import scala.collection.mutable import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.{ PartitionInfo, TopicPartition } -import org.apache.spark.annotation.Experimental import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index 1ad9963039b9e..133ab5e706411 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -25,7 +25,6 @@ import org.apache.kafka.clients.consumer.{ ConsumerConfig, ConsumerRecord } import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} -import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 62badff27573f..98ff6e3bef3f8 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -32,9 +32,14 @@ import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingContext } import org.apache.spark.streaming.dstream._ +/** + * :: Experimental :: + * Companion object for constructing Kafka streams and RDDs + */ @Experimental object KafkaUtils extends Logging { /** + * :: Experimental :: * Scala constructor for a batch-oriented interface for consuming from Kafka. * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. @@ -71,6 +76,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Java constructor for a batch-oriented interface for consuming from Kafka. * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. @@ -98,6 +104,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Scala constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number @@ -120,6 +127,7 @@ object KafkaUtils extends Logging { } /** + * :: Experimental :: * Java constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * @param keyClass Class of the keys in the Kafka records diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala index 7ab7ac41a09a2..422c8c5efbc92 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala @@ -23,35 +23,51 @@ import scala.collection.JavaConverters._ import org.apache.kafka.common.TopicPartition +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: * Choice of how to schedule consumers for a given TopicPartition on an executor. * Kafka 0.10 consumers prefetch messages, so it's important for performance * to keep cached consumers on appropriate executors, not recreate them for every partition. * Choice of location is only a preference, not an absolute; partitions may be scheduled elsewhere. */ +@Experimental sealed trait LocationStrategy /** + * :: Experimental :: * Use this only if your executors are on the same nodes as your Kafka brokers. */ +@Experimental case object PreferBrokers extends LocationStrategy { def create: PreferBrokers.type = this } /** + * :: Experimental :: * Use this in most cases, it will consistently distribute partitions across all executors. */ +@Experimental case object PreferConsistent extends LocationStrategy { def create: PreferConsistent.type = this } /** + * :: Experimental :: * Use this to place particular TopicPartitions on particular hosts if your load is uneven. * Any TopicPartition not specified in the map will use a consistent location. */ +@Experimental case class PreferFixed private(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy +/** + * :: Experimental :: + * Use this to place particular TopicPartitions on particular hosts if your load is uneven. + * Any TopicPartition not specified in the map will use a consistent location. + */ +@Experimental object PreferFixed { def apply(hostMap: collection.Map[TopicPartition, String]): PreferFixed = { PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala index 11375b81d1d14..6130b4091ce00 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.kafka import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition +import org.apache.spark.annotation.Experimental /** * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the @@ -37,6 +38,7 @@ trait HasOffsetRanges { } /** + * :: Experimental :: * Represents any object that can commit a collection of [[OffsetRange]]s. * The direct Kafka DStream implements this interface (see * [[KafkaUtils.createDirectStream]]). @@ -54,18 +56,25 @@ trait HasOffsetRanges { * }) * }}} */ +@Experimental trait CanCommitOffsets { /** + * :: Experimental :: * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * This is only needed if you intend to store offsets in Kafka, instead of your own store. * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. */ + @Experimental def commitAsync(offsetRanges: Array[OffsetRange]): Unit /** + * :: Experimental :: * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. + * This is only needed if you intend to store offsets in Kafka, instead of your own store. * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. * @param callback Only the most recently provided callback will be used at commit. */ + @Experimental def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit } From 4caca180895da1d8e87639d8c63cf9e86076b693 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 29 Jun 2016 19:55:14 -0500 Subject: [PATCH 36/37] [SPARK-12177][Streaming][Kafka] fix docstring --- .../spark/streaming/kafka/DirectKafkaInputDStream.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala index 324824b56cea2..81880c54f405b 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala @@ -37,7 +37,7 @@ import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} import org.apache.spark.streaming.scheduler.rate.RateEstimator /** - * A stream of [[KafkaRDD]] where + * A DStream where * each given Kafka topic/partition corresponds to an RDD partition. * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number * of messages @@ -49,10 +49,8 @@ import org.apache.spark.streaming.scheduler.rate.RateEstimator * configuration parameters. * Requires "bootstrap.servers" to be set with Kafka broker(s), * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param driverConsumer zero-argument function for you to construct a Kafka Consumer, - * and subscribe topics or assign partitions. - * This consumer will be used on the driver to query for offsets only, not messages. - * See Consumer doc + * @param consumerStrategy In most cases, pass in [[Subscribe]], + * see [[ConsumerStrategy]] for more details * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ From cffb0e0fb89808732c3ab3c1c7d83049549e2e2d Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 29 Jun 2016 20:09:15 -0500 Subject: [PATCH 37/37] [SPARK-12177][Streaming][Kafka] move package name to include kafka 0.10 version number, to disambiguate from the older connector --- .../streaming/{kafka => kafka010}/CachedKafkaConsumer.scala | 6 +++--- .../streaming/{kafka => kafka010}/ConsumerStrategy.scala | 2 +- .../{kafka => kafka010}/DirectKafkaInputDStream.scala | 2 +- .../spark/streaming/{kafka => kafka010}/KafkaRDD.scala | 4 ++-- .../streaming/{kafka => kafka010}/KafkaRDDPartition.scala | 4 ++-- .../streaming/{kafka => kafka010}/KafkaTestUtils.scala | 4 ++-- .../spark/streaming/{kafka => kafka010}/KafkaUtils.scala | 4 ++-- .../streaming/{kafka => kafka010}/LocationStrategy.scala | 2 +- .../spark/streaming/{kafka => kafka010}/OffsetRange.scala | 6 +++--- .../spark/streaming/{kafka => kafka010}/package-info.java | 4 ++-- .../spark/streaming/{kafka => kafka010}/package.scala | 2 +- .../{kafka => kafka010}/JavaConsumerStrategySuite.java | 2 +- .../{kafka => kafka010}/JavaDirectKafkaStreamSuite.java | 2 +- .../streaming/{kafka => kafka010}/JavaKafkaRDDSuite.java | 2 +- .../{kafka => kafka010}/JavaLocationStrategySuite.java | 2 +- .../{kafka => kafka010}/DirectKafkaStreamSuite.scala | 2 +- .../spark/streaming/{kafka => kafka010}/KafkaRDDSuite.scala | 2 +- 17 files changed, 26 insertions(+), 26 deletions(-) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/CachedKafkaConsumer.scala (98%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/ConsumerStrategy.scala (99%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/DirectKafkaInputDStream.scala (99%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/KafkaRDD.scala (99%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/KafkaRDDPartition.scala (95%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/KafkaTestUtils.scala (98%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/KafkaUtils.scala (98%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/LocationStrategy.scala (98%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/OffsetRange.scala (98%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/package-info.java (90%) rename external/kafka-0-10/src/main/scala/org/apache/spark/streaming/{kafka => kafka010}/package.scala (96%) rename external/kafka-0-10/src/test/java/org/apache/spark/streaming/{kafka => kafka010}/JavaConsumerStrategySuite.java (98%) rename external/kafka-0-10/src/test/java/org/apache/spark/streaming/{kafka => kafka010}/JavaDirectKafkaStreamSuite.java (99%) rename external/kafka-0-10/src/test/java/org/apache/spark/streaming/{kafka => kafka010}/JavaKafkaRDDSuite.java (98%) rename external/kafka-0-10/src/test/java/org/apache/spark/streaming/{kafka => kafka010}/JavaLocationStrategySuite.java (97%) rename external/kafka-0-10/src/test/scala/org/apache/spark/streaming/{kafka => kafka010}/DirectKafkaStreamSuite.scala (99%) rename external/kafka-0-10/src/test/scala/org/apache/spark/streaming/{kafka => kafka010}/KafkaRDDSuite.scala (99%) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala similarity index 98% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala index b48a28297c2f4..fa3ea6131a507 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/CachedKafkaConsumer.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/CachedKafkaConsumer.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.{ util => ju } @@ -31,7 +31,7 @@ import org.apache.spark.internal.Logging * Underlying consumer is not threadsafe, so neither is this, * but processing the same topicpartition and group id in multiple threads is usually bad anyway. */ -private[kafka] +private[kafka010] class CachedKafkaConsumer[K, V] private( val groupId: String, val topic: String, @@ -104,7 +104,7 @@ class CachedKafkaConsumer[K, V] private( } -private[kafka] +private[kafka010] object CachedKafkaConsumer extends Logging { private case class CacheKey(groupId: String, topic: String, partition: Int) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala similarity index 99% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index 23016bf664f6b..079a07dbc2bd0 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.{ util => ju } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala similarity index 99% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index 81880c54f405b..acd1841d5305c 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.{ util => ju } import java.util.concurrent.ConcurrentLinkedQueue diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala similarity index 99% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala index 133ab5e706411..c15c16344924f 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.{ util => ju } @@ -135,7 +135,7 @@ private[spark] class KafkaRDD[K, V]( .sortWith(compareExecutors) } - protected[kafka] def compareExecutors( + protected[kafka010] def compareExecutors( a: ExecutorCacheTaskLocation, b: ExecutorCacheTaskLocation): Boolean = if (a.host == b.host) { diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala similarity index 95% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala index ed416b9c0ba04..95569b109f30d 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDDPartition.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import org.apache.kafka.common.TopicPartition @@ -28,7 +28,7 @@ import org.apache.spark.Partition * @param fromOffset inclusive starting offset * @param untilOffset exclusive ending offset */ -private[kafka] +private[kafka010] class KafkaRDDPartition( val index: Int, val topic: String, diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala similarity index 98% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala index b047c5462b588..13c08430db6be 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.io.File import java.lang.{Integer => JInt} @@ -47,7 +47,7 @@ import org.apache.spark.util.Utils * * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. */ -private[kafka] class KafkaTestUtils extends Logging { +private[kafka010] class KafkaTestUtils extends Logging { // Zookeeper related configurations private val zkHost = "localhost" diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala similarity index 98% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala index 98ff6e3bef3f8..c0524990bc4dc 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.{ util => ju } @@ -153,7 +153,7 @@ object KafkaUtils extends Logging { /** * Tweak kafka params to prevent issues on executors */ - private[kafka] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { + private[kafka010] def fixKafkaParams(kafkaParams: ju.HashMap[String, Object]): Unit = { logWarning(s"overriding ${ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG} to false for executor") kafkaParams.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, false: java.lang.Boolean) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala similarity index 98% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala index 422c8c5efbc92..df620300eae21 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/LocationStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.{ util => ju } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala similarity index 98% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala index 6130b4091ce00..c66d3c9b8d229 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition @@ -144,10 +144,10 @@ object OffsetRange { new OffsetRange(topicPartition.topic, topicPartition.partition, fromOffset, untilOffset) /** this is to avoid ClassNotFoundException during checkpoint restore */ - private[kafka] + private[kafka010] type OffsetRangeTuple = (String, Int, Long, Long) - private[kafka] + private[kafka010] def apply(t: OffsetRangeTuple) = new OffsetRange(t._1, t._2, t._3, t._4) } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java similarity index 90% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java index f264c322fa50d..ebfcf8764a328 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package-info.java +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package-info.java @@ -16,6 +16,6 @@ */ /** - * Spark Integration for Kafka + * Spark Integration for Kafka 0.10 */ -package org.apache.spark.streaming.kafka; +package org.apache.spark.streaming.kafka010; diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala similarity index 96% rename from external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala rename to external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala index 4dfd41d602e8d..2bfc1e84d7ccd 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka/package.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/package.scala @@ -18,6 +18,6 @@ package org.apache.spark.streaming /** - * Spark Integration for Kafka + * Spark Integration for Kafka 0.10 */ package object kafka diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaConsumerStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java similarity index 98% rename from external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaConsumerStrategySuite.java rename to external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java index 2c97b76566b28..aba45f5de6782 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaConsumerStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaConsumerStrategySuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka; +package org.apache.spark.streaming.kafka010; import java.io.Serializable; import java.util.*; diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java similarity index 99% rename from external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java rename to external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java index 1fc1cdb52d3e3..e57ede7afaef4 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaDirectKafkaStreamSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka; +package org.apache.spark.streaming.kafka010; import java.io.Serializable; import java.util.*; diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java similarity index 98% rename from external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java rename to external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java index 15a445379b1a8..548ba134dcddf 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaKafkaRDDSuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka; +package org.apache.spark.streaming.kafka010; import java.io.Serializable; import java.util.HashMap; diff --git a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaLocationStrategySuite.java b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java similarity index 97% rename from external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaLocationStrategySuite.java rename to external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java index e638b6c5d960e..7873c09e1af85 100644 --- a/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka/JavaLocationStrategySuite.java +++ b/external/kafka-0-10/src/test/java/org/apache/spark/streaming/kafka010/JavaLocationStrategySuite.java @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka; +package org.apache.spark.streaming.kafka010; import java.io.Serializable; import java.util.*; diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala similarity index 99% rename from external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala rename to external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index cfb0f41fb3031..776d11ad2f648 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.io.File import java.util.{ Arrays, HashMap => JHashMap, Map => JMap } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala similarity index 99% rename from external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala rename to external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala index 7aa9dc4a2bc8b..3d2546ddd936d 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.streaming.kafka010 import java.{ util => ju }