-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-12177] Refactored some API in Kafka 0.10 to make public API simpler #13996
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from all commits
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,217 @@ | ||
| /* | ||
| * 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.kafka010; | ||
|
|
||
| import scala.collection.Map$; | ||
|
|
||
| import org.apache.kafka.clients.consumer.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. | ||
| */ | ||
| @Experimental | ||
| public abstract class ConsumerStrategy<K, V> { | ||
|
|
||
| /** | ||
| * Kafka <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> configuration | ||
| * parameters</a> to be used on executors. Requires "bootstrap.servers" to be set with Kafka | ||
| * broker(s) specified in host1:port1,host2:port2 form. | ||
| */ | ||
| public abstract java.util.Map<String,Object> executorKafkaParams(); | ||
|
|
||
| /** | ||
| * 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. | ||
| */ | ||
| public abstract Consumer<K,V> onStart( | ||
| java.util.Map<TopicPartition, Long> currentOffsets); | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Subscribe to a collection of topics. | ||
| * | ||
| * @param topics collection of topics to subscribe | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Subscribe( | ||
| scala.collection.Iterable<String> topics, | ||
| scala.collection.Map<String, Object> kafkaParams) { | ||
| return Subscribe(topics, kafkaParams, Map$.MODULE$.<TopicPartition, scala.Long>empty()); | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Subscribe to a collection of topics. | ||
| * | ||
| * @param topics collection of topics to subscribe | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.html#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Subscribe( | ||
| scala.collection.Iterable<String> topics, | ||
| scala.collection.Map<String, Object> kafkaParams, | ||
| scala.collection.Map<TopicPartition, scala.Long> offsets) { | ||
| return new SubscribeStrategy(topics, kafkaParams, offsets); | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Subscribe to a collection of topics. | ||
| * | ||
| * @param topics collection of topics to subscribe | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.html#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Subscribe( | ||
| java.util.Collection<String> topics, | ||
| java.util.Map<String, Object> kafkaParams) { | ||
| return Subscribe(topics, kafkaParams, java.util.Collections.<TopicPartition, Long>emptyMap()); | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Subscribe to a collection of topics. | ||
| * | ||
| * @param topics collection of topics to subscribe | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.html#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Subscribe( | ||
| java.util.Collection<String> topics, | ||
| java.util.Map<String, Object> kafkaParams, | ||
| java.util.Map<TopicPartition, Long> offsets) { | ||
| return new SubscribeStrategy(topics, kafkaParams, offsets); | ||
| } | ||
|
|
||
|
|
||
| /** | ||
| * :: Experimental :: Assign a fixed collection of TopicPartitions | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fix docs |
||
| * | ||
| * @param topicPartitions collection of TopicPartitions to assign | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Assign( | ||
| scala.collection.Iterable<TopicPartition> topicPartitions, | ||
| scala.collection.Map<String, Object> kafkaParams) { | ||
| return Assign(topicPartitions, kafkaParams, Map$.MODULE$.<TopicPartition, scala.Long>empty()); | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: Assign a fixed collection of TopicPartitions | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fix docs |
||
| * | ||
| * @param topicPartitions collection of TopicPartitions to assign | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Assign( | ||
| scala.collection.Iterable<TopicPartition> topicPartitions, | ||
| scala.collection.Map<String, Object> kafkaParams, | ||
| scala.collection.Map<TopicPartition, scala.Long> offsets) { | ||
| return new AssignStrategy(topicPartitions, kafkaParams, offsets); | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: Assign a fixed collection of TopicPartitions | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fix |
||
| * | ||
| * @param topicPartitions collection of TopicPartitions to assign | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Assign( | ||
| java.util.Collection<TopicPartition> topicPartitions, | ||
| java.util.Map<String, Object> kafkaParams) { | ||
| return Assign(topicPartitions, kafkaParams, java.util.Collections.<TopicPartition, Long>emptyMap()); | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: Assign a fixed collection of TopicPartitions | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. fix |
||
| * | ||
| * @param topicPartitions collection of TopicPartitions to assign | ||
| * @param kafkaParams Kafka <a | ||
| * href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| public static <Key, Value> ConsumerStrategy<Key, Value> Assign( | ||
| java.util.Collection<TopicPartition> topicPartitions, | ||
| java.util.Map<String, Object> kafkaParams, | ||
| java.util.Map<TopicPartition, Long> offsets) { | ||
| return new AssignStrategy(topicPartitions, kafkaParams, offsets); | ||
| } | ||
|
|
||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,61 @@ | ||
| /* | ||
| * 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.kafka010; | ||
|
|
||
| import java.util.Map; | ||
|
|
||
| import scala.collection.JavaConversions; | ||
|
|
||
| import org.apache.kafka.common.TopicPartition; | ||
| import org.apache.spark.annotation.Experimental; | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Abstract class representing 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 | ||
| public abstract class LocationStrategy { | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Use this in most cases, it will consistently distribute partitions across all executors. | ||
| */ | ||
| public static LocationStrategy PreferConsistent() { | ||
| return PreferConsistent$.MODULE$; | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Use this only if your executors are on the same nodes as your Kafka brokers. | ||
| */ | ||
| public static LocationStrategy PreferBrokers() { | ||
| return PreferBrokers$.MODULE$; | ||
| } | ||
|
|
||
|
|
||
| public static LocationStrategy PreferFixed(Map<TopicPartition, String> hostMap) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: docs? |
||
| return new PreferFixed(hostMap); | ||
| } | ||
|
|
||
| public static LocationStrategy PreferFixed(scala.collection.Map<TopicPartition, String> hostMap) { | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. nit: docs? |
||
| return PreferFixed(JavaConversions.mapAsJavaMap(hostMap)); | ||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,117 @@ | ||
| /* | ||
| * 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.kafka010 | ||
|
|
||
| import java.{ util => ju } | ||
|
|
||
| import scala.collection.JavaConverters._ | ||
|
|
||
| import org.apache.kafka.clients.consumer._ | ||
| import org.apache.kafka.common.TopicPartition | ||
|
|
||
| import org.apache.spark.annotation.Experimental | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Subscribe to a collection of topics. | ||
| * @param topics collection of topics to subscribe | ||
| * @param kafkaParams Kafka | ||
| * <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| private[kafka010] case class SubscribeStrategy[K, V] private( | ||
| topics: ju.Collection[String], | ||
| kafkaParams: ju.Map[String, Object], | ||
| offsets: ju.Map[TopicPartition, java.lang.Long] | ||
| ) extends ConsumerStrategy[K, V] { | ||
|
|
||
| def this( | ||
| topics: collection.Iterable[String], | ||
| kafkaParams: collection.Map[String, Object], | ||
| offsets: collection.Map[TopicPartition, scala.Long]) = this( | ||
| topics.asJavaCollection, | ||
| MapConverter.asJavaMap(kafkaParams), | ||
| MapConverter.asJavaMap(offsets.mapValues(l => new java.lang.Long(l)))) | ||
|
|
||
| def executorKafkaParams(): ju.Map[String, Object] = kafkaParams | ||
|
|
||
| def onStart(currentOffsets: ju.Map[TopicPartition, java.lang.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 | ||
| } | ||
| } | ||
|
|
||
| /** | ||
| * :: Experimental :: | ||
| * Assign a fixed collection of TopicPartitions | ||
| * @param topicPartitions collection of TopicPartitions to assign | ||
| * @param kafkaParams Kafka | ||
| * <a href="http://kafka.apache.org/documentation.htmll#newconsumerconfigs"> | ||
| * configuration parameters</a> 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. | ||
| */ | ||
| @Experimental | ||
| private[kafka010] case class AssignStrategy[K, V] private( | ||
| topicPartitions: ju.Collection[TopicPartition], | ||
| kafkaParams: ju.Map[String, Object], | ||
| offsets: ju.Map[TopicPartition, java.lang.Long] | ||
| ) extends ConsumerStrategy[K, V] { | ||
|
|
||
| def this( | ||
| topicPartitions: collection.Iterable[TopicPartition], | ||
| kafkaParams: collection.Map[String, Object], | ||
| offsets: collection.Map[TopicPartition, scala.Long]) = this( | ||
| topicPartitions.asJavaCollection, | ||
| MapConverter.asJavaMap(kafkaParams), | ||
| MapConverter.asJavaMap(offsets.mapValues(l => new java.lang.Long(l)))) | ||
|
|
||
| def executorKafkaParams(): ju.Map[String, Object] = kafkaParams | ||
|
|
||
| def onStart(currentOffsets: ju.Map[TopicPartition, java.lang.Long]): Consumer[K, V] = { | ||
| val consumer = new KafkaConsumer[K, V](kafkaParams) | ||
| consumer.assign(topicPartitions) | ||
| if (currentOffsets.isEmpty) { | ||
| offsets.asScala.foreach { case (topicPartition, offset) => | ||
| consumer.seek(topicPartition, offset) | ||
| } | ||
| } | ||
|
|
||
| consumer | ||
| } | ||
| } | ||
|
|
||
|
|
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
fix docs