-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-4062][Streaming]Add ReliableKafkaReceiver in Spark Streaming Kafka connector #2991
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 1 commit
dd9aeeb
77c3e50
0894aef
16bfe78
a949741
8135d31
96c7a1d
4854ee9
98f3d07
ea873e4
e5e21c1
b798535
e501b3c
b2b2f84
9f636b3
2a20a01
ec2e95e
d9a452c
14630aa
149948b
fab14c7
eae4ad6
5461f1c
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
- Loading branch information
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
| @@ -0,0 +1,135 @@ | ||
| /* | ||
| * 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.Map | ||
| import scala.reflect.{classTag, ClassTag} | ||
|
|
||
| import java.util.Properties | ||
| import java.util.concurrent.Executors | ||
|
|
||
| import kafka.consumer._ | ||
| import kafka.serializer.Decoder | ||
| import kafka.utils.VerifiableProperties | ||
| import kafka.utils.ZKStringSerializer | ||
| import org.I0Itec.zkclient._ | ||
|
|
||
| import org.apache.spark.Logging | ||
| import org.apache.spark.storage.StorageLevel | ||
| import org.apache.spark.streaming.receiver.Receiver | ||
|
|
||
| private[streaming] | ||
| class KafkaReceiver[ | ||
| K: ClassTag, | ||
| V: ClassTag, | ||
| U <: Decoder[_]: ClassTag, | ||
| T <: Decoder[_]: ClassTag]( | ||
| kafkaParams: Map[String, String], | ||
| topics: Map[String, Int], | ||
| storageLevel: StorageLevel | ||
| ) extends Receiver[Any](storageLevel) with Logging { | ||
|
||
|
|
||
| // Connection to Kafka | ||
| var consumerConnector: ConsumerConnector = null | ||
|
|
||
| def onStop() { | ||
| if (consumerConnector != null) { | ||
| consumerConnector.shutdown() | ||
| } | ||
| } | ||
|
|
||
| def onStart() { | ||
|
|
||
| logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) | ||
|
|
||
| // Kafka connection properties | ||
| val props = new Properties() | ||
| kafkaParams.foreach(param => props.put(param._1, param._2)) | ||
|
|
||
| val zkConnect = kafkaParams("zookeeper.connect") | ||
| // Create the connection to the cluster | ||
| logInfo("Connecting to Zookeeper: " + zkConnect) | ||
| val consumerConfig = new ConsumerConfig(props) | ||
| consumerConnector = Consumer.create(consumerConfig) | ||
| logInfo("Connected to " + zkConnect) | ||
|
|
||
| // When auto.offset.reset is defined, it is our responsibility to try and whack the | ||
| // consumer group zk node. | ||
| if (kafkaParams.contains("auto.offset.reset")) { | ||
| tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams("group.id")) | ||
| } | ||
|
|
||
| val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) | ||
| .newInstance(consumerConfig.props) | ||
| .asInstanceOf[Decoder[K]] | ||
| val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) | ||
| .newInstance(consumerConfig.props) | ||
| .asInstanceOf[Decoder[V]] | ||
|
|
||
| // Create Threads for each Topic/Message Stream we are listening | ||
|
||
| val topicMessageStreams = consumerConnector.createMessageStreams( | ||
| topics, keyDecoder, valueDecoder) | ||
|
|
||
| val executorPool = Executors.newFixedThreadPool(topics.values.sum) | ||
| try { | ||
| // Start the messages handler for each partition | ||
| topicMessageStreams.values.foreach { streams => | ||
| streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } | ||
| } | ||
| } finally { | ||
| executorPool.shutdown() // Just causes threads to terminate after work is done | ||
| } | ||
| } | ||
|
|
||
| // Handles Kafka Messages | ||
| private class MessageHandler[K: ClassTag, V: ClassTag](stream: KafkaStream[K, V]) | ||
| extends Runnable { | ||
| def run() { | ||
| logInfo("Starting MessageHandler.") | ||
| try { | ||
| for (msgAndMetadata <- stream) { | ||
|
||
| store((msgAndMetadata.key, msgAndMetadata.message)) | ||
| } | ||
| } catch { | ||
| case e: Throwable => logError("Error handling message; exiting", e) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| // It is our responsibility to delete the consumer group when specifying auto.offset.reset. This | ||
| // is because Kafka 0.7.2 only honors this param when the group is not in zookeeper. | ||
| // | ||
| // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied | ||
| // from Kafka's ConsoleConsumer. See code related to 'auto.offset.reset' when it is set to | ||
| // 'smallest'/'largest': | ||
| // scalastyle:off | ||
| // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala | ||
| // scalastyle:on | ||
| private def tryZookeeperConsumerGroupCleanup(zkUrl: String, groupId: String) { | ||
| val dir = "/consumers/" + groupId | ||
| logInfo("Cleaning up temporary Zookeeper data under " + dir + ".") | ||
|
||
| val zk = new ZkClient(zkUrl, 30*1000, 30*1000, ZKStringSerializer) | ||
| try { | ||
| zk.deleteRecursive(dir) | ||
| } catch { | ||
| case e: Throwable => logWarning("Error cleaning up temporary Zookeeper data", e) | ||
| } finally { | ||
| zk.close() | ||
| } | ||
| } | ||
| } | ||
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.
If you make the ReliableKafkaReceiver extend Receiver[(K, V)], then this
asInstanceOfis not necessary.