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