@@ -42,15 +42,12 @@ import org.apache.spark.storage.StorageLevel
4242import org .apache .spark .streaming .{Milliseconds , StreamingContext }
4343import org .apache .spark .util .Utils
4444
45- abstract class KafkaStreamSuiteBase extends FunSuite with Logging {
46- import KafkaTestUtils ._
45+ /**
46+ * This is an abstract base class for Kafka testsuites. This has the functionality to set up
47+ * and tear down local Kafka servers, and to push data using Kafka producers.
48+ */
49+ abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging {
4750
48- val sparkConf = new SparkConf ()
49- .setMaster(" local[4]" )
50- .setAppName(this .getClass.getSimpleName)
51- val batchDuration = Milliseconds (500 )
52- var ssc : StreamingContext = _
53-
5451 var zkAddress : String = _
5552 var zkClient : ZkClient = _
5653
@@ -64,7 +61,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging {
6461 private var server : KafkaServer = _
6562 private var producer : Producer [String , String ] = _
6663
67- def beforeFunction () {
64+ def setupKafka () {
6865 // Zookeeper server startup
6966 zookeeper = new EmbeddedZookeeper (s " $zkHost: $zkPort" )
7067 // Get the actual zookeeper binding port
@@ -80,7 +77,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging {
8077 var bindSuccess : Boolean = false
8178 while (! bindSuccess) {
8279 try {
83- val brokerProps = getBrokerConfig(brokerPort, zkAddress )
80+ val brokerProps = getBrokerConfig()
8481 brokerConf = new KafkaConfig (brokerProps)
8582 server = new KafkaServer (brokerConf)
8683 logInfo(" ==================== 2 ====================" )
@@ -100,12 +97,7 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging {
10097 logInfo(" ==================== 4 ====================" )
10198 }
10299
103- def afterFunction () {
104- if (ssc != null ) {
105- ssc.stop()
106- ssc = null
107- }
108-
100+ def tearDownKafka () {
109101 if (producer != null ) {
110102 producer.close()
111103 producer = null
@@ -141,101 +133,43 @@ abstract class KafkaStreamSuiteBase extends FunSuite with Logging {
141133 CreateTopicCommand .createTopic(zkClient, topic, 1 , 1 , " 0" )
142134 logInfo(" ==================== 5 ====================" )
143135 // wait until metadata is propagated
144- waitUntilMetadataIsPropagated(Seq (server), topic, 0 , 1000 )
136+ waitUntilMetadataIsPropagated(topic, 0 )
145137 }
146138
147139 def produceAndSendMessage (topic : String , sent : Map [String , Int ]) {
148- val brokerAddr = brokerConf.hostName + " :" + brokerConf.port
149- if (producer == null ) {
150- producer = new Producer [String , String ](new ProducerConfig (getProducerConfig(brokerAddr)))
151- }
140+ producer = new Producer [String , String ](new ProducerConfig (getProducerConfig()))
152141 producer.send(createTestMessage(topic, sent): _* )
142+ producer.close()
153143 logInfo(" ==================== 6 ====================" )
154144 }
155- }
156-
157- class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually {
158-
159- before { beforeFunction() }
160- after { afterFunction() }
161-
162- test(" Kafka input stream" ) {
163- ssc = new StreamingContext (sparkConf, batchDuration)
164- val topic = " topic1"
165- val sent = Map (" a" -> 5 , " b" -> 3 , " c" -> 10 )
166- createTopic(topic)
167- produceAndSendMessage(topic, sent)
168-
169- val kafkaParams = Map (" zookeeper.connect" -> zkAddress,
170- " group.id" -> s " test-consumer- ${Random .nextInt(10000 )}" ,
171- " auto.offset.reset" -> " smallest" )
172-
173- val stream = KafkaUtils .createStream[String , String , StringDecoder , StringDecoder ](
174- ssc,
175- kafkaParams,
176- Map (topic -> 1 ),
177- StorageLevel .MEMORY_ONLY )
178- val result = new mutable.HashMap [String , Long ]()
179- stream.map { case (k, v) => v }
180- .countByValue()
181- .foreachRDD { r =>
182- val ret = r.collect()
183- ret.toMap.foreach { kv =>
184- val count = result.getOrElseUpdate(kv._1, 0 ) + kv._2
185- result.put(kv._1, count)
186- }
187- }
188- ssc.start()
189- eventually(timeout(3000 milliseconds), interval(100 milliseconds)) {
190- assert(sent.size === result.size)
191- sent.keys.foreach { k => assert(sent(k) === result(k).toInt) }
192- }
193-
194- ssc.stop()
195- }
196- }
197-
198145
199- object KafkaTestUtils {
200-
201- def getBrokerConfig (port : Int , zkConnect : String ): Properties = {
146+ private def getBrokerConfig (): Properties = {
202147 val props = new Properties ()
203148 props.put(" broker.id" , " 0" )
204149 props.put(" host.name" , " localhost" )
205- props.put(" port" , port .toString)
150+ props.put(" port" , brokerPort .toString)
206151 props.put(" log.dir" , Utils .createTempDir().getAbsolutePath)
207- props.put(" zookeeper.connect" , zkConnect )
152+ props.put(" zookeeper.connect" , zkAddress )
208153 props.put(" log.flush.interval.messages" , " 1" )
209154 props.put(" replica.socket.timeout.ms" , " 1500" )
210155 props
211156 }
212157
213- def getProducerConfig (brokerList : String ): Properties = {
158+ private def getProducerConfig (): Properties = {
159+ val brokerAddr = brokerConf.hostName + " :" + brokerConf.port
214160 val props = new Properties ()
215- props.put(" metadata.broker.list" , brokerList )
161+ props.put(" metadata.broker.list" , brokerAddr )
216162 props.put(" serializer.class" , classOf [StringEncoder ].getName)
217163 props
218164 }
219165
220- def waitUntilTrue (condition : () => Boolean , waitTime : Long ): Boolean = {
221- val startTime = System .currentTimeMillis()
222- while (true ) {
223- if (condition())
224- return true
225- if (System .currentTimeMillis() > startTime + waitTime)
226- return false
227- Thread .sleep(waitTime.min(100L ))
166+ private def waitUntilMetadataIsPropagated (topic : String , partition : Int ) {
167+ eventually(timeout(1000 milliseconds), interval(100 milliseconds)) {
168+ assert(
169+ server.apis.leaderCache.keySet.contains(TopicAndPartition (topic, partition)),
170+ s " Partition [ $topic, $partition] metadata not propagated after timeout "
171+ )
228172 }
229- // Should never go to here
230- throw new RuntimeException (" unexpected error" )
231- }
232-
233- def waitUntilMetadataIsPropagated (servers : Seq [KafkaServer ], topic : String , partition : Int ,
234- timeout : Long ) {
235- assert(waitUntilTrue(() =>
236- servers.foldLeft(true )(_ && _.apis.leaderCache.keySet.contains(
237- TopicAndPartition (topic, partition))), timeout),
238- s " Partition [ $topic, $partition] metadata not propagated after timeout " )
239173 }
240174
241175 class EmbeddedZookeeper (val zkConnect : String ) {
@@ -261,3 +195,53 @@ object KafkaTestUtils {
261195 }
262196 }
263197}
198+
199+
200+ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter {
201+ var ssc : StreamingContext = _
202+
203+ before {
204+ setupKafka()
205+ }
206+
207+ after {
208+ if (ssc != null ) {
209+ ssc.stop()
210+ ssc = null
211+ }
212+ tearDownKafka()
213+ }
214+
215+ test(" Kafka input stream" ) {
216+ val sparkConf = new SparkConf ().setMaster(" local[4]" ).setAppName(this .getClass.getSimpleName)
217+ ssc = new StreamingContext (sparkConf, Milliseconds (500 ))
218+ val topic = " topic1"
219+ val sent = Map (" a" -> 5 , " b" -> 3 , " c" -> 10 )
220+ createTopic(topic)
221+ produceAndSendMessage(topic, sent)
222+
223+ val kafkaParams = Map (" zookeeper.connect" -> zkAddress,
224+ " group.id" -> s " test-consumer- ${Random .nextInt(10000 )}" ,
225+ " auto.offset.reset" -> " smallest" )
226+
227+ val stream = KafkaUtils .createStream[String , String , StringDecoder , StringDecoder ](
228+ ssc, kafkaParams, Map (topic -> 1 ), StorageLevel .MEMORY_ONLY )
229+ val result = new mutable.HashMap [String , Long ]()
230+ stream.map(_._2).countByValue().foreachRDD { r =>
231+ val ret = r.collect()
232+ ret.toMap.foreach { kv =>
233+ val count = result.getOrElseUpdate(kv._1, 0 ) + kv._2
234+ result.put(kv._1, count)
235+ }
236+ }
237+ ssc.start()
238+ eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
239+ assert(sent.size === result.size)
240+ sent.keys.foreach { k =>
241+ assert(sent(k) === result(k).toInt)
242+ }
243+ }
244+ ssc.stop()
245+ }
246+ }
247+
0 commit comments