1+ /*
2+ * Licensed to the Apache Software Foundation (ASF) under one or more
3+ * contributor license agreements. See the NOTICE file distributed with
4+ * this work for additional information regarding copyright ownership.
5+ * The ASF licenses this file to You under the Apache License, Version 2.0
6+ * (the "License"); you may not use this file except in compliance with
7+ * the License. You may obtain a copy of the License at
8+ *
9+ * http://www.apache.org/licenses/LICENSE-2.0
10+ *
11+ * Unless required by applicable law or agreed to in writing, software
12+ * distributed under the License is distributed on an "AS IS" BASIS,
13+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+ * See the License for the specific language governing permissions and
15+ * limitations under the License.
16+ */
17+
18+ package org .apache .spark .streaming .kafka
19+
20+ import scala .collection .Map
21+ import scala .reflect .{classTag , ClassTag }
22+
23+ import java .util .Properties
24+ import java .util .concurrent .Executors
25+
26+ import kafka .consumer ._
27+ import kafka .serializer .Decoder
28+ import kafka .utils .VerifiableProperties
29+ import kafka .utils .ZKStringSerializer
30+ import org .I0Itec .zkclient ._
31+
32+ import org .apache .spark .Logging
33+ import org .apache .spark .storage .StorageLevel
34+ import org .apache .spark .streaming .receiver .Receiver
35+
36+ private [streaming]
37+ class KafkaReceiver [
38+ K : ClassTag ,
39+ V : ClassTag ,
40+ U <: Decoder [_]: ClassTag ,
41+ T <: Decoder [_]: ClassTag ](
42+ kafkaParams : Map [String , String ],
43+ topics : Map [String , Int ],
44+ storageLevel : StorageLevel
45+ ) extends Receiver [Any ](storageLevel) with Logging {
46+
47+ // Connection to Kafka
48+ var consumerConnector : ConsumerConnector = null
49+
50+ def onStop () {
51+ if (consumerConnector != null ) {
52+ consumerConnector.shutdown()
53+ }
54+ }
55+
56+ def onStart () {
57+
58+ logInfo(" Starting Kafka Consumer Stream with group: " + kafkaParams(" group.id" ))
59+
60+ // Kafka connection properties
61+ val props = new Properties ()
62+ kafkaParams.foreach(param => props.put(param._1, param._2))
63+
64+ val zkConnect = kafkaParams(" zookeeper.connect" )
65+ // Create the connection to the cluster
66+ logInfo(" Connecting to Zookeeper: " + zkConnect)
67+ val consumerConfig = new ConsumerConfig (props)
68+ consumerConnector = Consumer .create(consumerConfig)
69+ logInfo(" Connected to " + zkConnect)
70+
71+ // When auto.offset.reset is defined, it is our responsibility to try and whack the
72+ // consumer group zk node.
73+ if (kafkaParams.contains(" auto.offset.reset" )) {
74+ tryZookeeperConsumerGroupCleanup(zkConnect, kafkaParams(" group.id" ))
75+ }
76+
77+ val keyDecoder = classTag[U ].runtimeClass.getConstructor(classOf [VerifiableProperties ])
78+ .newInstance(consumerConfig.props)
79+ .asInstanceOf [Decoder [K ]]
80+ val valueDecoder = classTag[T ].runtimeClass.getConstructor(classOf [VerifiableProperties ])
81+ .newInstance(consumerConfig.props)
82+ .asInstanceOf [Decoder [V ]]
83+
84+ // Create Threads for each Topic/Message Stream we are listening
85+ val topicMessageStreams = consumerConnector.createMessageStreams(
86+ topics, keyDecoder, valueDecoder)
87+
88+ val executorPool = Executors .newFixedThreadPool(topics.values.sum)
89+ try {
90+ // Start the messages handler for each partition
91+ topicMessageStreams.values.foreach { streams =>
92+ streams.foreach { stream => executorPool.submit(new MessageHandler (stream)) }
93+ }
94+ } finally {
95+ executorPool.shutdown() // Just causes threads to terminate after work is done
96+ }
97+ }
98+
99+ // Handles Kafka Messages
100+ private class MessageHandler [K : ClassTag , V : ClassTag ](stream : KafkaStream [K , V ])
101+ extends Runnable {
102+ def run () {
103+ logInfo(" Starting MessageHandler." )
104+ try {
105+ for (msgAndMetadata <- stream) {
106+ store((msgAndMetadata.key, msgAndMetadata.message))
107+ }
108+ } catch {
109+ case e : Throwable => logError(" Error handling message; exiting" , e)
110+ }
111+ }
112+ }
113+
114+ // It is our responsibility to delete the consumer group when specifying auto.offset.reset. This
115+ // is because Kafka 0.7.2 only honors this param when the group is not in zookeeper.
116+ //
117+ // The kafka high level consumer doesn't expose setting offsets currently, this is a trick copied
118+ // from Kafka's ConsoleConsumer. See code related to 'auto.offset.reset' when it is set to
119+ // 'smallest'/'largest':
120+ // scalastyle:off
121+ // https://github.com/apache/kafka/blob/0.7.2/core/src/main/scala/kafka/consumer/ConsoleConsumer.scala
122+ // scalastyle:on
123+ private def tryZookeeperConsumerGroupCleanup (zkUrl : String , groupId : String ) {
124+ val dir = " /consumers/" + groupId
125+ logInfo(" Cleaning up temporary Zookeeper data under " + dir + " ." )
126+ val zk = new ZkClient (zkUrl, 30 * 1000 , 30 * 1000 , ZKStringSerializer )
127+ try {
128+ zk.deleteRecursive(dir)
129+ } catch {
130+ case e : Throwable => logWarning(" Error cleaning up temporary Zookeeper data" , e)
131+ } finally {
132+ zk.close()
133+ }
134+ }
135+ }
0 commit comments