├── src
├── test
│ ├── resources
│ │ ├── testDataDir
│ │ │ └── testdata
│ │ ├── zookeeper.properties
│ │ └── kafka-server.properties
│ └── scala
│ │ └── org
│ │ └── cloudera
│ │ └── spark
│ │ └── streaming
│ │ └── kafka
│ │ ├── util
│ │ ├── KafkaLocal.scala
│ │ ├── ZooKeeperLocal.scala
│ │ ├── KafkaConsumer.scala
│ │ └── TestUtil.scala
│ │ └── TestKafkaOutputDStream.scala
└── main
│ └── scala
│ └── org
│ └── cloudera
│ └── spark
│ └── streaming
│ └── kafka
│ ├── ProducerObject.scala
│ └── KafkaWriter.scala
├── .travis.yml
├── .gitignore
├── pom.xml
└── LICENSE
/src/test/resources/testDataDir/testdata:
--------------------------------------------------------------------------------
1 |
--------------------------------------------------------------------------------
/.travis.yml:
--------------------------------------------------------------------------------
1 | language: scala
2 | jdk:
3 | - oraclejdk7
4 | script: mvn clean install
5 |
--------------------------------------------------------------------------------
/.gitignore:
--------------------------------------------------------------------------------
1 | *.class
2 | *.log
3 |
4 | # sbt specific
5 | .cache/
6 | .history/
7 | .lib/
8 | dist/*
9 | target/
10 | lib_managed/
11 | src_managed/
12 | project/boot/
13 | project/plugins/project/
14 | .idea/
15 | # Scala-IDE specific
16 | .scala_dependencies
17 | .worksheet
18 |
--------------------------------------------------------------------------------
/src/test/resources/zookeeper.properties:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 | # the directory where the snapshot is stored.
16 | dataDir=target
17 | # the port at which the clients will connect
18 | clientPort=2181
19 | # disable the per-ip limit on the number of connections since this is a non-production config
20 | maxClientCnxns=0
--------------------------------------------------------------------------------
/src/main/scala/org/cloudera/spark/streaming/kafka/ProducerObject.scala:
--------------------------------------------------------------------------------
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 | package org.cloudera.spark.streaming.kafka
18 |
19 | object ProducerObject {
20 |
21 | private var producerOpt: Option[Any] = None
22 |
23 | def getCachedProducer: Any = {
24 | producerOpt.get
25 | }
26 |
27 | def cacheProducer(producer: Any): Unit = {
28 | producerOpt = Some(producer)
29 | }
30 |
31 | def isCached: Boolean = {
32 | producerOpt.isDefined
33 | }
34 | }
35 |
--------------------------------------------------------------------------------
/src/test/scala/org/cloudera/spark/streaming/kafka/util/KafkaLocal.scala:
--------------------------------------------------------------------------------
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 | limitations under the License.
17 | */
18 | package org.apache.spark.streaming.kafka.util
19 |
20 | import kafka.server.KafkaConfig
21 | import kafka.server.KafkaServerStartable
22 | import java.util.Properties
23 |
24 |
25 | /**
26 | * A local Kafka server for running unit tests.
27 | * Reference: https://gist.github.com/fjavieralba/7930018/
28 | */
29 | class KafkaLocal(kafkaProperties: Properties) {
30 | private val kafkaConfig: KafkaConfig = new KafkaConfig(kafkaProperties)
31 | private val kafka = new KafkaServerStartable(kafkaConfig)
32 |
33 | def start() {
34 | kafka.startup()
35 | }
36 |
37 | def stop() {
38 | kafka.shutdown()
39 | }
40 | }
--------------------------------------------------------------------------------
/src/test/scala/org/cloudera/spark/streaming/kafka/util/ZooKeeperLocal.scala:
--------------------------------------------------------------------------------
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 | limitations under the License.
17 | */
18 | package org.apache.spark.streaming.kafka.util
19 |
20 | import org.apache.zookeeper.server.ServerConfig
21 | import org.apache.zookeeper.server.ZooKeeperServerMain
22 | import org.apache.zookeeper.server.quorum.QuorumPeerConfig
23 | import org.slf4j.Logger
24 | import org.slf4j.LoggerFactory
25 | import java.io.IOException
26 | import java.util.Properties
27 |
28 | /**
29 | * A local Zookeeper server for running unit tests.
30 | * Reference: https://gist.github.com/fjavieralba/7930018/
31 | */
32 | class ZooKeeperLocal(zkProperties: Properties) {
33 | private val quorumConfiguration: QuorumPeerConfig = new QuorumPeerConfig
34 | private final val logger: Logger = LoggerFactory.getLogger(classOf[ZooKeeperLocal])
35 | private val zooKeeperServer: ZooKeeperServerMain = new ZooKeeperServerMain
36 | private val configuration: ServerConfig = new ServerConfig
37 | try {
38 | quorumConfiguration.parseProperties(zkProperties)
39 | }
40 | catch {
41 | case e: Exception =>
42 | throw new RuntimeException(e)
43 | }
44 | configuration.readFrom(quorumConfiguration)
45 | new Thread {
46 | override def run() {
47 | try {
48 | zooKeeperServer.runFromConfig(configuration)
49 | }
50 | catch {
51 | case e: IOException =>
52 | logger.error("Zookeeper startup failed.", e)
53 | }
54 | }
55 | }.start()
56 | }
57 |
58 |
--------------------------------------------------------------------------------
/src/test/scala/org/cloudera/spark/streaming/kafka/TestKafkaOutputDStream.scala:
--------------------------------------------------------------------------------
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 | package org.cloudera.spark.streaming.kafka
18 |
19 | import java.util.Properties
20 |
21 | import kafka.producer.KeyedMessage
22 | import org.apache.spark.SparkConf
23 | import org.apache.spark.rdd.RDD
24 | import org.apache.spark.streaming.{Duration, StreamingContext}
25 | import org.apache.spark.streaming.kafka.util.TestUtil
26 | import org.junit.{Before, Test, Assert}
27 | import org.cloudera.spark.streaming.kafka.KafkaWriter._
28 |
29 | import scala.collection.mutable
30 | import scala.collection.mutable.ArrayBuffer
31 |
32 | class TestKafkaOutputDStream {
33 | private val testUtil: TestUtil = TestUtil.getInstance
34 | // Name of the framework for Spark context
35 | def framework = this.getClass.getSimpleName
36 |
37 | // Master for Spark context
38 | def master = "local[2]"
39 | val conf = new SparkConf()
40 | .setMaster(master)
41 | .setAppName(framework)
42 | // conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock")
43 | val ssc = new StreamingContext(conf, Duration.apply(2000))
44 |
45 | @Before def setup() {
46 | testUtil.prepare()
47 | val topics: java.util.List[String] = new java.util.ArrayList[String](3)
48 | topics.add("default")
49 | topics.add("static")
50 | topics.add("custom")
51 | testUtil.initTopicList(topics)
52 | }
53 |
54 | @Test
55 | def testKafkaDStream(): Unit = {
56 | val toBe = new mutable.Queue[RDD[String]]()
57 | var j = 0
58 | while (j < 9) {
59 | toBe.enqueue(ssc.sparkContext.makeRDD(Seq(j.toString, (j + 1).toString, (j + 2).toString)))
60 | j += 3
61 | }
62 | val instream = ssc.queueStream(toBe)
63 | val producerConf = new Properties()
64 | producerConf.put("serializer.class", "kafka.serializer.DefaultEncoder")
65 | producerConf.put("key.serializer.class", "kafka.serializer.StringEncoder")
66 | producerConf.put("metadata.broker.list", testUtil.getKafkaServerUrl)
67 | producerConf.put("request.required.acks", "1")
68 | instream.writeToKafka(producerConf,
69 | (x: String) => new KeyedMessage[String,Array[Byte]]("default", null,x.getBytes))
70 | ssc.start()
71 |
72 | Thread.sleep(10000)
73 | var i = 0
74 | val expectedResults = (0 to 8).map(_.toString).toSeq
75 | val actualResults = new ArrayBuffer[String]()
76 | while (i < 9) {
77 | println(i)
78 | val fetchedMsg = new String(
79 | testUtil.getNextMessageFromConsumer("default").message.asInstanceOf[Array[Byte]])
80 | Assert.assertNotNull(fetchedMsg)
81 | actualResults += fetchedMsg
82 | i += 1
83 | }
84 | val actualResultSorted = actualResults.sorted
85 | println(actualResultSorted.mkString)
86 | Assert.assertEquals(expectedResults.toSeq, actualResultSorted.toSeq)
87 | }
88 | }
89 |
--------------------------------------------------------------------------------
/src/test/scala/org/cloudera/spark/streaming/kafka/util/KafkaConsumer.scala:
--------------------------------------------------------------------------------
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 | limitations under the License.
17 | */
18 | package org.apache.spark.streaming.kafka.util
19 |
20 | import kafka.consumer.ConsumerConfig
21 | import kafka.consumer.ConsumerIterator
22 | import kafka.consumer.ConsumerTimeoutException
23 | import kafka.consumer.KafkaStream
24 | import kafka.message.MessageAndMetadata
25 |
26 | import org.slf4j.Logger
27 | import org.slf4j.LoggerFactory
28 |
29 | import java.util.HashMap
30 | import java.util.List
31 | import java.util.Map
32 | import java.util.Properties
33 |
34 | import scala.collection.JavaConversions._
35 |
36 | /**
37 | * A Kafka Consumer implementation. This uses the current thread to fetch the
38 | * next message from the queue and doesn't use a multi threaded implementation.
39 | * So this implements a synchronous blocking call.
40 | * To avoid infinite waiting, a timeout is implemented to wait only for
41 | * 10 seconds before concluding that the message will not be available.
42 | */
43 | object KafkaConsumer {
44 | private def createConsumerConfig(zkUrl: String, groupId: String): ConsumerConfig = {
45 | val props: Properties = new Properties
46 | props.put("zookeeper.connect", zkUrl)
47 | props.put("group.id", groupId)
48 | props.put("zookeeper.session.timeout.ms", "400")
49 | props.put("zookeeper.sync.time.ms", "200")
50 | props.put("auto.commit.interval.ms", "1000")
51 | props.put("auto.offset.reset", "smallest")
52 | props.put("consumer.timeout.ms", "1000")
53 | new ConsumerConfig(props)
54 | }
55 |
56 | }
57 |
58 | class KafkaConsumer (val consumer: kafka.consumer.ConsumerConnector = kafka.consumer.Consumer.create(
59 | KafkaConsumer.createConsumerConfig(TestUtil.getInstance
60 | .getZkUrl, "group_1"))) {
61 |
62 | private[util] var consumerMap: Map[String, scala.List[KafkaStream[Array[Byte],
63 | Array[Byte]]]] = null
64 |
65 | private final val logger: Logger = LoggerFactory.getLogger(classOf[KafkaConsumer])
66 |
67 | def initTopicList(topics: List[String]) {
68 | val topicCountMap: Map[String, Int] = new HashMap[String, Int]
69 | for (topic <- topics) {
70 | topicCountMap.put(topic, new Integer(1))
71 | }
72 | consumerMap = consumer.createMessageStreams(topicCountMap)
73 | }
74 |
75 | def getNextMessage(topic: String): MessageAndMetadata[_, _] = {
76 | val streams: scala.List[KafkaStream[Array[Byte], Array[Byte]]] = consumerMap.get(topic)
77 | val stream: KafkaStream[Array[Byte], Array[Byte]] = streams.get(0)
78 | val it: ConsumerIterator[Array[Byte], Array[Byte]] = stream.iterator()
79 | try {
80 | if (it.hasNext()) {
81 | it.next()
82 | }
83 | else {
84 | null
85 | }
86 | }
87 | catch {
88 | case e: ConsumerTimeoutException => {
89 | logger.error("0 messages available to fetch for the topic " + topic)
90 | null
91 | }
92 | }
93 | }
94 |
95 | def shutdown(): Unit = {
96 | consumer.shutdown()
97 | }
98 | }
99 |
100 |
--------------------------------------------------------------------------------
/pom.xml:
--------------------------------------------------------------------------------
1 |
2 |
5 | 4.0.0
6 |
7 | org.cloudera.spark.streaming.kafka
8 | spark-kafka-writer
9 | 0.1-SNAPSHOT
10 | jar
11 |
12 |
13 |
14 | org.apache.spark
15 | spark-streaming_2.10
16 | 1.1.0
17 |
18 |
19 | org.apache.kafka
20 | kafka_2.10
21 | 0.8.0
22 |
23 |
24 | com.sun.jmx
25 | jmxri
26 |
27 |
28 | com.sun.jdmk
29 | jmxtools
30 |
31 |
32 | javax.jms
33 | jms
34 |
35 |
36 |
37 |
38 | junit
39 | junit
40 | 4.11
41 | test
42 |
43 |
44 | org.scalacheck
45 | scalacheck_2.10
46 | 1.11.3
47 | test
48 |
49 |
50 |
51 |
52 | sonatype-nexus-snapshots
53 | Sonatype Nexus Snapshots
54 |
55 | https://oss.sonatype.org/content/repositories/snapshots
56 |
57 |
58 | false
59 |
60 |
61 | true
62 |
63 |
64 |
65 |
66 |
67 | sonatype-nexus-snapshots
68 | Sonatype Nexus Snapshots
69 | https://oss.sonatype.org/content/repositories/snapshots/
70 |
71 |
72 | sonatype-nexus-staging
73 | Nexus Release Repository
74 |
75 | https://oss.sonatype.org/service/local/staging/deploy/maven2/
76 |
77 |
78 |
79 |
80 |
81 |
82 |
83 | org.apache.maven.plugins
84 | maven-release-plugin
85 | 2.4.2
86 |
87 |
88 |
89 |
90 |
91 | org.apache.maven.plugins
92 | maven-compiler-plugin
93 | 2.3.2
94 |
95 |
96 | org.apache.maven.plugins
97 | maven-surefire-plugin
98 | 2.17
99 |
100 | always
101 | 900
102 | true
103 |
104 | **/Test*.java
105 |
106 |
107 |
108 |
109 | org.scala-tools
110 | maven-scala-plugin
111 |
112 |
113 |
114 | compile
115 | testCompile
116 |
117 |
118 |
119 |
120 | src/main/scala
121 |
122 | -Xms64m
123 | -Xmx1024m
124 |
125 |
126 |
127 |
128 |
129 |
130 |
131 | sign
132 |
133 |
134 |
135 | org.apache.maven.plugins
136 | maven-gpg-plugin
137 |
138 |
139 | verify
140 |
141 | sign
142 |
143 |
144 |
145 |
146 |
147 |
148 |
149 |
150 |
151 |
152 |
--------------------------------------------------------------------------------
/src/main/scala/org/cloudera/spark/streaming/kafka/KafkaWriter.scala:
--------------------------------------------------------------------------------
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.cloudera.spark.streaming.kafka
19 |
20 | import java.util.Properties
21 |
22 | import kafka.producer.{ProducerConfig, KeyedMessage, Producer}
23 | import org.apache.spark.rdd.RDD
24 | import org.apache.spark.streaming.dstream.DStream
25 |
26 | import scala.reflect.ClassTag
27 |
28 | /**
29 | * Import this object in this form:
30 | * {{{
31 | * import org.apache.spark.streaming.kafka.KafkaWriter._
32 | * }}}
33 | *
34 | * Once this is done, the `writeToKafka` can be called on the [[DStream]] object in this form:
35 | * {{{
36 | * dstream.writeToKafka(producerConfig, f)
37 | * }}}
38 | */
39 | object KafkaWriter {
40 | /**
41 | * This implicit method allows the user to call dstream.writeToKafka(..)
42 | * @param dstream - DStream to write to Kafka
43 | * @tparam T - The type of the DStream
44 | * @tparam K - The type of the key to serialize to
45 | * @tparam V - The type of the value to serialize to
46 | * @return
47 | */
48 | implicit def createKafkaOutputWriter[T: ClassTag, K, V](
49 | dstream: DStream[T]): KafkaWriter[T] = {
50 | new KafkaWriter[T](dstream)
51 | }
52 | }
53 |
54 | /**
55 | *
56 | * This class can be used to write data to Kafka from Spark Streaming. To write data to Kafka
57 | * simply `import org.apache.spark.streaming.kafka.KafkaWriter._` in your application and call
58 | * `dstream.writeToKafka(producerConf, func)`
59 | *
60 | * Here is an example:
61 | * {{{
62 | * // Adding this line allows the user to call dstream.writeDStreamToKafka(..)
63 | * import org.apache.spark.streaming.kafka.KafkaWriter._
64 | *
65 | * class ExampleWriter {
66 | * val instream = ssc.queueStream(toBe)
67 | * val producerConf = new Properties()
68 | * producerConf.put("serializer.class", "kafka.serializer.DefaultEncoder")
69 | * producerConf.put("key.serializer.class", "kafka.serializer.StringEncoder")
70 | * producerConf.put("metadata.broker.list", "kafka.example.com:5545")
71 | * producerConf.put("request.required.acks", "1")
72 | * instream.writeToKafka(producerConf,
73 | * (x: String) => new KeyedMessage[String,String]("default", null, x))
74 | * ssc.start()
75 | * }
76 | *
77 | * }}}
78 | * @param dstream - The [[DStream]] to be written to Kafka
79 | *
80 | */
81 | class KafkaWriter[T: ClassTag](@transient dstream: DStream[T]) {
82 |
83 | /**
84 | * To write data from a DStream to Kafka, call this function after creating the DStream. Once
85 | * the DStream is passed into this function, all data coming from the DStream is written out to
86 | * Kafka. The properties instance takes the configuration required to connect to the Kafka
87 | * brokers in the standard Kafka format. The serializerFunc is a function that converts each
88 | * element of the RDD to a Kafka [[KeyedMessage]]. This closure should be serializable - so it
89 | * should use only instances of Serializables.
90 | * @param producerConfig The configuration that can be used to connect to Kafka
91 | * @param serializerFunc The function to convert the data from the stream into Kafka
92 | * [[KeyedMessage]]s.
93 | * @tparam K The type of the key
94 | * @tparam V The type of the value
95 | *
96 | */
97 | def writeToKafka[K, V](producerConfig: Properties,
98 | serializerFunc: T => KeyedMessage[K, V]): Unit = {
99 | // Broadcast the producer to avoid sending it every time.
100 | val broadcastedConfig = dstream.context.sparkContext.broadcast(producerConfig)
101 | def func = (rdd: RDD[T]) => {
102 | rdd.foreachPartition(events => {
103 | // The ForEachDStream runs the function locally on the driver. So the
104 | // ProducerObject from the driver is likely to get serialized and
105 | // sent, which is fine - because at that point the Producer itself is
106 | // not initialized, so a None is sent over the wire.
107 | // Get the producer from that local executor and write!
108 | val producer: Producer[K, V] = {
109 | if (ProducerObject.isCached) {
110 | ProducerObject.getCachedProducer
111 | .asInstanceOf[Producer[K, V]]
112 | } else {
113 | val producer =
114 | new Producer[K, V](new ProducerConfig(broadcastedConfig.value))
115 | ProducerObject.cacheProducer(producer)
116 | producer
117 | }
118 | }
119 | producer.send(events.map(serializerFunc).toArray: _*)
120 | })
121 | }
122 | dstream.foreachRDD(func)
123 | }
124 |
125 | }
126 |
--------------------------------------------------------------------------------
/src/test/scala/org/cloudera/spark/streaming/kafka/util/TestUtil.scala:
--------------------------------------------------------------------------------
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 | limitations under the License.
17 | */
18 | package org.apache.spark.streaming.kafka.util
19 |
20 | import java.net.{BindException, InetAddress, UnknownHostException}
21 | import java.util.{List, Properties, Random}
22 |
23 | import kafka.message.MessageAndMetadata
24 | import org.slf4j.{Logger, LoggerFactory}
25 |
26 | /**
27 | * A utility class for starting/stopping Kafka Server.
28 | */
29 | object TestUtil {
30 |
31 | private val instance: TestUtil = new TestUtil()
32 |
33 | def getInstance: TestUtil = {
34 | instance
35 | }
36 | }
37 |
38 | class TestUtil {
39 | init()
40 | private val randPortGen: Random = new Random(System.currentTimeMillis)
41 | private var kafkaServer: KafkaLocal = null
42 | private var kafkaConsumer: KafkaConsumer = null
43 | private var hostname: String = "localhost"
44 | private var kafkaLocalPort: Int = 0
45 | private var zkLocalPort: Int = 0
46 | private final val logger: Logger = LoggerFactory.getLogger(classOf[TestUtil])
47 |
48 | private def init() {
49 | try {
50 | hostname = InetAddress.getLocalHost.getHostName
51 | }
52 | catch {
53 | case e: UnknownHostException => {
54 | logger.warn("Error getting the value of localhost. " + "Proceeding with 'localhost'.", e)
55 | }
56 | }
57 | }
58 |
59 | private def startKafkaServer: Boolean = {
60 | val kafkaProperties: Properties = new Properties
61 | val zkProperties: Properties = new Properties
62 | try {
63 | zkProperties.load(classOf[Class[_]].getResourceAsStream("/zookeeper.properties"))
64 | var zookeeper: ZooKeeperLocal = null
65 | var portAssigned = false
66 | while (!portAssigned) {
67 | try {
68 | zkLocalPort = getNextPort
69 | zkProperties.setProperty("clientPort", Integer.toString(zkLocalPort))
70 | zookeeper = new ZooKeeperLocal(zkProperties)
71 | portAssigned = true
72 | }
73 | catch {
74 | case bindEx: BindException => {
75 | }
76 | }
77 | }
78 | logger.info("ZooKeeper instance is successfully started on port " + zkLocalPort)
79 | kafkaProperties.load(classOf[Class[_]].getResourceAsStream("/kafka-server.properties"))
80 | kafkaProperties.setProperty("zookeeper.connect", getZkUrl)
81 | var started = false
82 | while (!started) {
83 | kafkaLocalPort = getNextPort
84 | kafkaProperties.setProperty("port", Integer.toString(kafkaLocalPort))
85 | kafkaServer = new KafkaLocal(kafkaProperties)
86 | try {
87 | kafkaServer.start()
88 | started = true
89 | }
90 | catch {
91 | case bindEx: BindException => {
92 | }
93 | }
94 | }
95 | logger.info("Kafka Server is successfully started on port " + kafkaLocalPort)
96 | true
97 | }
98 | catch {
99 | case e: Exception => {
100 | logger.error("Error starting the Kafka Server.", e)
101 | false
102 | }
103 | }
104 | }
105 |
106 | private def getKafkaConsumer: KafkaConsumer = {
107 | this synchronized {
108 | if (kafkaConsumer == null) {
109 | kafkaConsumer = new KafkaConsumer
110 | }
111 | }
112 | kafkaConsumer
113 | }
114 |
115 | def initTopicList(topics: List[String]) {
116 | getKafkaConsumer.initTopicList(topics)
117 | }
118 |
119 | def getNextMessageFromConsumer(topic: String): MessageAndMetadata[_, _] = {
120 | getKafkaConsumer.getNextMessage(topic)
121 | }
122 |
123 | def prepare() {
124 | val startStatus: Boolean = startKafkaServer
125 | if (!startStatus) {
126 | throw new RuntimeException("Error starting the server!")
127 | }
128 | try {
129 | Thread.sleep(3 * 1000)
130 | }
131 | catch {
132 | case e: InterruptedException => {
133 | }
134 | }
135 | getKafkaConsumer
136 | logger.info("Completed the prepare phase.")
137 | }
138 |
139 | def tearDown() {
140 | logger.info("Shutting down the Kafka Consumer.")
141 | getKafkaConsumer.shutdown()
142 | try {
143 | Thread.sleep(3 * 1000)
144 | }
145 | catch {
146 | case e: InterruptedException => {
147 | }
148 | }
149 | logger.info("Shutting down the kafka Server.")
150 | kafkaServer.stop()
151 | logger.info("Completed the tearDown phase.")
152 | }
153 |
154 | private def getNextPort: Int = {
155 | randPortGen.nextInt(65535 - 49152) + 49152
156 | }
157 |
158 | def getZkUrl: String = {
159 | hostname + ":" + zkLocalPort
160 | }
161 |
162 | def getKafkaServerUrl: String = {
163 | hostname + ":" + kafkaLocalPort
164 | }
165 | }
166 |
167 |
--------------------------------------------------------------------------------
/src/test/resources/kafka-server.properties:
--------------------------------------------------------------------------------
1 | # Licensed to the Apache Software Foundation (ASF) under one or more
2 | # contributor license agreements. See the NOTICE file distributed with
3 | # this work for additional information regarding copyright ownership.
4 | # The ASF licenses this file to You under the Apache License, Version 2.0
5 | # (the "License"); you may not use this file except in compliance with
6 | # the License. You may obtain a copy of the License at
7 | #
8 | # http://www.apache.org/licenses/LICENSE-2.0
9 | #
10 | # Unless required by applicable law or agreed to in writing, software
11 | # distributed under the License is distributed on an "AS IS" BASIS,
12 | # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 | # See the License for the specific language governing permissions and
14 | # limitations under the License.
15 | # see kafka.server.KafkaConfig for additional details and defaults
16 |
17 | ############################# Server Basics #############################
18 |
19 | # The id of the broker. This must be set to a unique integer for each broker.
20 | broker.id=0
21 |
22 | ############################# Socket Server Settings #############################
23 |
24 | # The port the socket server listens on
25 | port=9092
26 |
27 | # Hostname the broker will bind to. If not set, the server will bind to all interfaces
28 | #host.name=localhost
29 |
30 | # Hostname the broker will advertise to producers and consumers. If not set, it uses the
31 | # value for "host.name" if configured. Otherwise, it will use the value returned from
32 | # java.net.InetAddress.getCanonicalHostName().
33 | #advertised.host.name=
34 |
35 | # The port to publish to ZooKeeper for clients to use. If this is not set,
36 | # it will publish the same port that the broker binds to.
37 | #advertised.port=
38 |
39 | # The number of threads handling network requests
40 | num.network.threads=2
41 |
42 | # The number of threads doing disk I/O
43 | num.io.threads=8
44 |
45 | # The send buffer (SO_SNDBUF) used by the socket server
46 | socket.send.buffer.bytes=1048576
47 |
48 | # The receive buffer (SO_RCVBUF) used by the socket server
49 | socket.receive.buffer.bytes=1048576
50 |
51 | # The maximum size of a request that the socket server will accept (protection against OOM)
52 | socket.request.max.bytes=104857600
53 |
54 |
55 | ############################# Log Basics #############################
56 |
57 | # A comma seperated list of directories under which to store log files
58 | log.dirs=target/kafka-logs
59 |
60 | # The default number of log partitions per topic. More partitions allow greater
61 | # parallelism for consumption, but this will also result in more files across
62 | # the brokers.
63 | num.partitions=2
64 |
65 | ############################# Log Flush Policy #############################
66 |
67 | # Messages are immediately written to the filesystem but by default we only fsync() to sync
68 | # the OS cache lazily. The following configurations control the flush of data to disk.
69 | # There are a few important trade-offs here:
70 | # 1. Durability: Unflushed data may be lost if you are not using replication.
71 | # 2. Latency: Very large flush intervals may lead to latency spikes when the flush does occur as there will be a lot of data to flush.
72 | # 3. Throughput: The flush is generally the most expensive operation, and a small flush interval may lead to exceessive seeks.
73 | # The settings below allow one to configure the flush policy to flush data after a period of time or
74 | # every N messages (or both). This can be done globally and overridden on a per-topic basis.
75 |
76 | # The number of messages to accept before forcing a flush of data to disk
77 | #log.flush.interval.messages=10000
78 |
79 | # The maximum amount of time a message can sit in a log before we force a flush
80 | #log.flush.interval.ms=1000
81 |
82 | ############################# Log Retention Policy #############################
83 |
84 | # The following configurations control the disposal of log segments. The policy can
85 | # be set to delete segments after a period of time, or after a given size has accumulated.
86 | # A segment will be deleted whenever *either* of these criteria are met. Deletion always happens
87 | # from the end of the log.
88 |
89 | # The minimum age of a log file to be eligible for deletion
90 | log.retention.hours=168
91 |
92 | # A size-based retention policy for logs. Segments are pruned from the log as long as the remaining
93 | # segments don't drop below log.retention.bytes.
94 | #log.retention.bytes=1073741824
95 |
96 | # The maximum size of a log segment file. When this size is reached a new log segment will be created.
97 | log.segment.bytes=536870912
98 |
99 | # The interval at which log segments are checked to see if they can be deleted according
100 | # to the retention policies
101 | log.retention.check.interval.ms=60000
102 |
103 | # By default the log cleaner is disabled and the log retention policy will default to just delete segments after their retention expires.
104 | # If log.cleaner.enable=true is set the cleaner will be enabled and individual logs can then be marked for log compaction.
105 | log.cleaner.enable=false
106 |
107 | ############################# Zookeeper #############################
108 |
109 | # Zookeeper connection string (see zookeeper docs for details).
110 | # This is a comma separated host:port pairs, each corresponding to a zk
111 | # server. e.g. "127.0.0.1:3000,127.0.0.1:3001,127.0.0.1:3002".
112 | # You can also append an optional chroot string to the urls to specify the
113 | # root directory for all kafka znodes.
114 | zookeeper.connect=localhost:2181
115 |
116 | # Timeout in ms for connecting to zookeeper
117 | zookeeper.connection.timeout.ms=1000000
118 |
--------------------------------------------------------------------------------
/LICENSE:
--------------------------------------------------------------------------------
1 | Apache License
2 | Version 2.0, January 2004
3 | http://www.apache.org/licenses/
4 |
5 | TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
6 |
7 | 1. Definitions.
8 |
9 | "License" shall mean the terms and conditions for use, reproduction,
10 | and distribution as defined by Sections 1 through 9 of this document.
11 |
12 | "Licensor" shall mean the copyright owner or entity authorized by
13 | the copyright owner that is granting the License.
14 |
15 | "Legal Entity" shall mean the union of the acting entity and all
16 | other entities that control, are controlled by, or are under common
17 | control with that entity. For the purposes of this definition,
18 | "control" means (i) the power, direct or indirect, to cause the
19 | direction or management of such entity, whether by contract or
20 | otherwise, or (ii) ownership of fifty percent (50%) or more of the
21 | outstanding shares, or (iii) beneficial ownership of such entity.
22 |
23 | "You" (or "Your") shall mean an individual or Legal Entity
24 | exercising permissions granted by this License.
25 |
26 | "Source" form shall mean the preferred form for making modifications,
27 | including but not limited to software source code, documentation
28 | source, and configuration files.
29 |
30 | "Object" form shall mean any form resulting from mechanical
31 | transformation or translation of a Source form, including but
32 | not limited to compiled object code, generated documentation,
33 | and conversions to other media types.
34 |
35 | "Work" shall mean the work of authorship, whether in Source or
36 | Object form, made available under the License, as indicated by a
37 | copyright notice that is included in or attached to the work
38 | (an example is provided in the Appendix below).
39 |
40 | "Derivative Works" shall mean any work, whether in Source or Object
41 | form, that is based on (or derived from) the Work and for which the
42 | editorial revisions, annotations, elaborations, or other modifications
43 | represent, as a whole, an original work of authorship. For the purposes
44 | of this License, Derivative Works shall not include works that remain
45 | separable from, or merely link (or bind by name) to the interfaces of,
46 | the Work and Derivative Works thereof.
47 |
48 | "Contribution" shall mean any work of authorship, including
49 | the original version of the Work and any modifications or additions
50 | to that Work or Derivative Works thereof, that is intentionally
51 | submitted to Licensor for inclusion in the Work by the copyright owner
52 | or by an individual or Legal Entity authorized to submit on behalf of
53 | the copyright owner. For the purposes of this definition, "submitted"
54 | means any form of electronic, verbal, or written communication sent
55 | to the Licensor or its representatives, including but not limited to
56 | communication on electronic mailing lists, source code control systems,
57 | and issue tracking systems that are managed by, or on behalf of, the
58 | Licensor for the purpose of discussing and improving the Work, but
59 | excluding communication that is conspicuously marked or otherwise
60 | designated in writing by the copyright owner as "Not a Contribution."
61 |
62 | "Contributor" shall mean Licensor and any individual or Legal Entity
63 | on behalf of whom a Contribution has been received by Licensor and
64 | subsequently incorporated within the Work.
65 |
66 | 2. Grant of Copyright License. Subject to the terms and conditions of
67 | this License, each Contributor hereby grants to You a perpetual,
68 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
69 | copyright license to reproduce, prepare Derivative Works of,
70 | publicly display, publicly perform, sublicense, and distribute the
71 | Work and such Derivative Works in Source or Object form.
72 |
73 | 3. Grant of Patent License. Subject to the terms and conditions of
74 | this License, each Contributor hereby grants to You a perpetual,
75 | worldwide, non-exclusive, no-charge, royalty-free, irrevocable
76 | (except as stated in this section) patent license to make, have made,
77 | use, offer to sell, sell, import, and otherwise transfer the Work,
78 | where such license applies only to those patent claims licensable
79 | by such Contributor that are necessarily infringed by their
80 | Contribution(s) alone or by combination of their Contribution(s)
81 | with the Work to which such Contribution(s) was submitted. If You
82 | institute patent litigation against any entity (including a
83 | cross-claim or counterclaim in a lawsuit) alleging that the Work
84 | or a Contribution incorporated within the Work constitutes direct
85 | or contributory patent infringement, then any patent licenses
86 | granted to You under this License for that Work shall terminate
87 | as of the date such litigation is filed.
88 |
89 | 4. Redistribution. You may reproduce and distribute copies of the
90 | Work or Derivative Works thereof in any medium, with or without
91 | modifications, and in Source or Object form, provided that You
92 | meet the following conditions:
93 |
94 | (a) You must give any other recipients of the Work or
95 | Derivative Works a copy of this License; and
96 |
97 | (b) You must cause any modified files to carry prominent notices
98 | stating that You changed the files; and
99 |
100 | (c) You must retain, in the Source form of any Derivative Works
101 | that You distribute, all copyright, patent, trademark, and
102 | attribution notices from the Source form of the Work,
103 | excluding those notices that do not pertain to any part of
104 | the Derivative Works; and
105 |
106 | (d) If the Work includes a "NOTICE" text file as part of its
107 | distribution, then any Derivative Works that You distribute must
108 | include a readable copy of the attribution notices contained
109 | within such NOTICE file, excluding those notices that do not
110 | pertain to any part of the Derivative Works, in at least one
111 | of the following places: within a NOTICE text file distributed
112 | as part of the Derivative Works; within the Source form or
113 | documentation, if provided along with the Derivative Works; or,
114 | within a display generated by the Derivative Works, if and
115 | wherever such third-party notices normally appear. The contents
116 | of the NOTICE file are for informational purposes only and
117 | do not modify the License. You may add Your own attribution
118 | notices within Derivative Works that You distribute, alongside
119 | or as an addendum to the NOTICE text from the Work, provided
120 | that such additional attribution notices cannot be construed
121 | as modifying the License.
122 |
123 | You may add Your own copyright statement to Your modifications and
124 | may provide additional or different license terms and conditions
125 | for use, reproduction, or distribution of Your modifications, or
126 | for any such Derivative Works as a whole, provided Your use,
127 | reproduction, and distribution of the Work otherwise complies with
128 | the conditions stated in this License.
129 |
130 | 5. Submission of Contributions. Unless You explicitly state otherwise,
131 | any Contribution intentionally submitted for inclusion in the Work
132 | by You to the Licensor shall be under the terms and conditions of
133 | this License, without any additional terms or conditions.
134 | Notwithstanding the above, nothing herein shall supersede or modify
135 | the terms of any separate license agreement you may have executed
136 | with Licensor regarding such Contributions.
137 |
138 | 6. Trademarks. This License does not grant permission to use the trade
139 | names, trademarks, service marks, or product names of the Licensor,
140 | except as required for reasonable and customary use in describing the
141 | origin of the Work and reproducing the content of the NOTICE file.
142 |
143 | 7. Disclaimer of Warranty. Unless required by applicable law or
144 | agreed to in writing, Licensor provides the Work (and each
145 | Contributor provides its Contributions) on an "AS IS" BASIS,
146 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
147 | implied, including, without limitation, any warranties or conditions
148 | of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
149 | PARTICULAR PURPOSE. You are solely responsible for determining the
150 | appropriateness of using or redistributing the Work and assume any
151 | risks associated with Your exercise of permissions under this License.
152 |
153 | 8. Limitation of Liability. In no event and under no legal theory,
154 | whether in tort (including negligence), contract, or otherwise,
155 | unless required by applicable law (such as deliberate and grossly
156 | negligent acts) or agreed to in writing, shall any Contributor be
157 | liable to You for damages, including any direct, indirect, special,
158 | incidental, or consequential damages of any character arising as a
159 | result of this License or out of the use or inability to use the
160 | Work (including but not limited to damages for loss of goodwill,
161 | work stoppage, computer failure or malfunction, or any and all
162 | other commercial damages or losses), even if such Contributor
163 | has been advised of the possibility of such damages.
164 |
165 | 9. Accepting Warranty or Additional Liability. While redistributing
166 | the Work or Derivative Works thereof, You may choose to offer,
167 | and charge a fee for, acceptance of support, warranty, indemnity,
168 | or other liability obligations and/or rights consistent with this
169 | License. However, in accepting such obligations, You may act only
170 | on Your own behalf and on Your sole responsibility, not on behalf
171 | of any other Contributor, and only if You agree to indemnify,
172 | defend, and hold each Contributor harmless for any liability
173 | incurred by, or claims asserted against, such Contributor by reason
174 | of your accepting any such warranty or additional liability.
175 |
176 | END OF TERMS AND CONDITIONS
177 |
178 | APPENDIX: How to apply the Apache License to your work.
179 |
180 | To apply the Apache License to your work, attach the following
181 | boilerplate notice, with the fields enclosed by brackets "{}"
182 | replaced with your own identifying information. (Don't include
183 | the brackets!) The text should be enclosed in the appropriate
184 | comment syntax for the file format. We also recommend that a
185 | file or class name and description of purpose be included on the
186 | same "printed page" as the copyright notice for easier
187 | identification within third-party archives.
188 |
189 | Copyright {yyyy} {name of copyright owner}
190 |
191 | Licensed under the Apache License, Version 2.0 (the "License");
192 | you may not use this file except in compliance with the License.
193 | You may obtain a copy of the License at
194 |
195 | http://www.apache.org/licenses/LICENSE-2.0
196 |
197 | Unless required by applicable law or agreed to in writing, software
198 | distributed under the License is distributed on an "AS IS" BASIS,
199 | WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
200 | See the License for the specific language governing permissions and
201 | limitations under the License.
202 |
203 |
--------------------------------------------------------------------------------