From 658687b25491047f30ee8558733d11e5a0572070 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Wed, 11 Feb 2015 00:13:27 -0800 Subject: [SPARK-4964] [Streaming] refactor createRDD to take leaders via map instead of array Author: cody koeninger Closes #4511 from koeninger/kafkaRdd-leader-to-broker and squashes the following commits: f7151d4 [cody koeninger] [SPARK-4964] test refactoring 6f8680b [cody koeninger] [SPARK-4964] add test of the scala api for KafkaUtils.createRDD f81e016 [cody koeninger] [SPARK-4964] leave KafkaStreamSuite host and port as private 5173f3f [cody koeninger] [SPARK-4964] test the Java variations of createRDD e9cece4 [cody koeninger] [SPARK-4964] pass leaders as a map to ensure 1 leader per TopicPartition --- .../org/apache/spark/streaming/kafka/Broker.scala | 68 ++++++++++++++++++++++ .../apache/spark/streaming/kafka/KafkaUtils.scala | 44 +++++++++----- .../org/apache/spark/streaming/kafka/Leader.scala | 57 ------------------ 3 files changed, 98 insertions(+), 71 deletions(-) create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala delete mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala (limited to 'external/kafka/src/main') diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala new file mode 100644 index 0000000000..5a74febb4b --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala @@ -0,0 +1,68 @@ +/* + * 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 + +/** + * :: Experimental :: + * Represent the host and port info for a Kafka broker. + * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID + */ +@Experimental +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/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 7a2c3abdcc..af04bc6576 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -154,6 +154,19 @@ object KafkaUtils { jssc.ssc, kafkaParams.toMap, Map(topics.mapValues(_.intValue()).toSeq: _*), storageLevel) } + /** get leaders for the given offset ranges, or throw an exception */ + private def leadersForRanges( + kafkaParams: Map[String, String], + offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { + val kc = new KafkaCluster(kafkaParams) + val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet + val leaders = kc.findLeaders(topics).fold( + errs => throw new SparkException(errs.mkString("\n")), + ok => ok + ) + leaders + } + /** * Create a RDD from Kafka using offset ranges for each topic and partition. * @@ -176,12 +189,7 @@ object KafkaUtils { offsetRanges: Array[OffsetRange] ): RDD[(K, V)] = { val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet - val leaders = kc.findLeaders(topics).fold( - errs => throw new SparkException(errs.mkString("\n")), - ok => ok - ) + val leaders = leadersForRanges(kafkaParams, offsetRanges) new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) } @@ -198,7 +206,8 @@ object KafkaUtils { * 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 leaders for each offset range in batch + * @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 */ @Experimental @@ -211,12 +220,17 @@ object KafkaUtils { sc: SparkContext, kafkaParams: Map[String, String], offsetRanges: Array[OffsetRange], - leaders: Array[Leader], + leaders: Map[TopicAndPartition, Broker], messageHandler: MessageAndMetadata[K, V] => R ): RDD[R] = { - val leaderMap = leaders - .map(l => TopicAndPartition(l.topic, l.partition) -> (l.host, l.port)) - .toMap + val leaderMap = if (leaders.isEmpty) { + leadersForRanges(kafkaParams, 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)) + }.toMap + } new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, messageHandler) } @@ -263,7 +277,8 @@ object KafkaUtils { * 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 leaders for each offset range in batch + * @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 */ @Experimental @@ -276,7 +291,7 @@ object KafkaUtils { recordClass: Class[R], kafkaParams: JMap[String, String], offsetRanges: Array[OffsetRange], - leaders: Array[Leader], + leaders: JMap[TopicAndPartition, Broker], messageHandler: JFunction[MessageAndMetadata[K, V], R] ): JavaRDD[R] = { implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) @@ -284,8 +299,9 @@ object KafkaUtils { 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.toSeq: _*) createRDD[K, V, KD, VD, R]( - jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges, leaders, messageHandler.call _) + jsc.sc, Map(kafkaParams.toSeq: _*), offsetRanges, leaderMap, messageHandler.call _) } /** diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala deleted file mode 100644 index c129a26836..0000000000 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/Leader.scala +++ /dev/null @@ -1,57 +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 kafka.common.TopicAndPartition - -import org.apache.spark.annotation.Experimental - -/** - * :: Experimental :: - * Represent the host info for the leader of a Kafka partition. - */ -@Experimental -final class Leader private( - /** Kafka topic name */ - val topic: String, - /** Kafka partition id */ - val partition: Int, - /** Leader's hostname */ - val host: String, - /** Leader's port */ - val port: Int) extends Serializable - -/** - * :: Experimental :: - * Companion object the provides methods to create instances of [[Leader]]. - */ -@Experimental -object Leader { - def create(topic: String, partition: Int, host: String, port: Int): Leader = - new Leader(topic, partition, host, port) - - def create(topicAndPartition: TopicAndPartition, host: String, port: Int): Leader = - new Leader(topicAndPartition.topic, topicAndPartition.partition, host, port) - - def apply(topic: String, partition: Int, host: String, port: Int): Leader = - new Leader(topic, partition, host, port) - - def apply(topicAndPartition: TopicAndPartition, host: String, port: Int): Leader = - new Leader(topicAndPartition.topic, topicAndPartition.partition, host, port) - -} -- cgit v1.2.3