1717
1818package org .apache .spark .streaming .kafka
1919
20+
2021import java .io .File
2122
2223import scala .collection .mutable
2324import scala .concurrent .duration ._
2425import scala .language .postfixOps
2526import scala .util .Random
2627
28+ import com .google .common .io .Files
2729import kafka .serializer .StringDecoder
2830import kafka .utils .{ZKGroupTopicDirs , ZkUtils }
31+ import org .apache .commons .io .FileUtils
2932import org .scalatest .BeforeAndAfter
3033import org .scalatest .concurrent .Eventually
3134
35+ import org .apache .spark .SparkConf
3236import org .apache .spark .storage .StorageLevel
33- import org .apache .spark .streaming .StreamingContext
34- import org .apache .spark .util .Utils
37+ import org .apache .spark .streaming .{Milliseconds , StreamingContext }
3538
3639class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually {
37- val topic = " topic"
40+
41+ val sparkConf = new SparkConf ()
42+ .setMaster(" local[4]" )
43+ .setAppName(this .getClass.getSimpleName)
44+ .set(" spark.streaming.receiver.writeAheadLog.enable" , " true" )
3845 val data = Map (" a" -> 10 , " b" -> 10 , " c" -> 10 )
46+
47+ var topic : String = _
3948 var groupId : String = _
4049 var kafkaParams : Map [String , String ] = _
50+ var ssc : StreamingContext = _
51+ var tempDirectory : File = null
4152
4253 before {
43- beforeFunction() // call this first to start ZK and Kafka
54+ setupKafka()
55+ topic = s " test-topic- ${Random .nextInt(10000 )}"
4456 groupId = s " test-consumer- ${Random .nextInt(10000 )}"
4557 kafkaParams = Map (
4658 " zookeeper.connect" -> zkAddress,
4759 " group.id" -> groupId,
4860 " auto.offset.reset" -> " smallest"
4961 )
62+
63+ ssc = new StreamingContext (sparkConf, Milliseconds (500 ))
64+ tempDirectory = Files .createTempDir()
65+ ssc.checkpoint(tempDirectory.getAbsolutePath)
5066 }
5167
5268 after {
53- afterFunction()
69+ if (ssc != null ) {
70+ ssc.stop()
71+ }
72+ if (tempDirectory != null && tempDirectory.exists()) {
73+ FileUtils .deleteDirectory(tempDirectory)
74+ tempDirectory = null
75+ }
76+ tearDownKafka()
5477 }
5578
56- test(" Reliable Kafka input stream" ) {
57- sparkConf.set(" spark.streaming.receiver.writeAheadLog.enable" , " true" )
58- ssc = new StreamingContext (sparkConf, batchDuration)
59- val checkpointDir = s " ${System .getProperty(" java.io.tmpdir" , " /tmp" )}/ " +
60- s " test-checkpoint ${Random .nextInt(10000 )}"
61- Utils .registerShutdownDeleteDir(new File (checkpointDir))
62- ssc.checkpoint(checkpointDir)
79+
80+ test(" Reliable Kafka input stream with single topic" ) {
6381 createTopic(topic)
6482 produceAndSendMessage(topic, data)
6583
84+ // Verify whether the offset of this group/topic/partition is 0 before starting.
85+ assert(getCommitOffset(groupId, topic, 0 ) === None )
86+
6687 val stream = KafkaUtils .createStream[String , String , StringDecoder , StringDecoder ](
67- ssc,
68- kafkaParams,
69- Map (topic -> 1 ),
70- StorageLevel .MEMORY_ONLY )
88+ ssc, kafkaParams, Map (topic -> 1 ), StorageLevel .MEMORY_ONLY )
7189 val result = new mutable.HashMap [String , Long ]()
7290 stream.map { case (k, v) => v }.foreachRDD { r =>
7391 val ret = r.collect()
@@ -77,84 +95,64 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter
7795 }
7896 }
7997 ssc.start()
80- eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
98+
99+ eventually(timeout(20000 milliseconds), interval(200 milliseconds)) {
81100 // A basic process verification for ReliableKafkaReceiver.
82101 // Verify whether received message number is equal to the sent message number.
83102 assert(data.size === result.size)
84103 // Verify whether each message is the same as the data to be verified.
85104 data.keys.foreach { k => assert(data(k) === result(k).toInt) }
105+ // Verify the offset number whether it is equal to the total message number.
106+ assert(getCommitOffset(groupId, topic, 0 ) === Some (29L ))
107+
86108 }
87109 ssc.stop()
88110 }
111+ /*
89112 test("Verify the offset commit") {
90113 // Verify the correctness of offset commit mechanism.
91- sparkConf.set(" spark.streaming.receiver.writeAheadLog.enable" , " true" )
92- ssc = new StreamingContext (sparkConf, batchDuration)
93- val checkpointDir = s " ${System .getProperty(" java.io.tmpdir" , " /tmp" )}/ " +
94- s " test-checkpoint ${Random .nextInt(10000 )}"
95- Utils .registerShutdownDeleteDir(new File (checkpointDir))
96- ssc.checkpoint(checkpointDir)
97-
98114 createTopic(topic)
99115 produceAndSendMessage(topic, data)
100116
101- // Verify whether the offset of this group/topic/partition is 0 before starting.
102- assert(getCommitOffset(groupId, topic, 0 ) === 0L )
103-
104117 // Do this to consume all the message of this group/topic.
105118 val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder](
106- ssc,
107- kafkaParams,
108- Map (topic -> 1 ),
109- StorageLevel .MEMORY_ONLY )
119+ ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY)
110120 stream.foreachRDD(_ => Unit)
111121 ssc.start()
112- eventually(timeout(3000 milliseconds), interval(100 milliseconds)) {
113- // Verify the offset number whether it is equal to the total message number.
114- assert(getCommitOffset(groupId, topic, 0 ) === 29L )
122+ eventually(timeout(20000 milliseconds), interval(200 milliseconds)) {
115123 }
116124 ssc.stop()
117125 }
118-
119- test(" Verify multiple topics offset commit" ) {
120- sparkConf.set(" spark.streaming.receiver.writeAheadLog.enable" , " true" )
121- ssc = new StreamingContext (sparkConf, batchDuration)
122- val checkpointDir = s " ${System .getProperty(" java.io.tmpdir" , " /tmp" )}/ " +
123- s " test-checkpoint ${Random .nextInt(10000 )}"
124- Utils .registerShutdownDeleteDir(new File (checkpointDir))
125- ssc.checkpoint(checkpointDir)
126-
126+ */
127+ test(" Reliable Kafka input stream with multiple topics" ) {
127128 val topics = Map (" topic1" -> 1 , " topic2" -> 1 , " topic3" -> 1 )
128129 topics.foreach { case (t, _) =>
129130 createTopic(t)
130131 produceAndSendMessage(t, data)
131132 }
132133
133134 // Before started, verify all the group/topic/partition offsets are 0.
134- topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0 ) === 0L ) }
135+ topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0 ) === None ) }
135136
136137 // Consuming all the data sent to the broker which will potential commit the offsets internally.
137138 val stream = KafkaUtils .createStream[String , String , StringDecoder , StringDecoder ](
138- ssc,
139- kafkaParams,
140- topics,
141- StorageLevel .MEMORY_ONLY )
139+ ssc, kafkaParams, topics, StorageLevel .MEMORY_ONLY )
142140 stream.foreachRDD(_ => Unit )
143141 ssc.start()
144- eventually(timeout(3000 milliseconds), interval(100 milliseconds)) {
142+ eventually(timeout(20000 milliseconds), interval(100 milliseconds)) {
145143 // Verify the offset for each group/topic to see whether they are equal to the expected one.
146- topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0 ) === 29L ) }
144+ topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0 ) === Some ( 29L ) ) }
147145 }
148146 ssc.stop()
149147 }
150148
149+
151150 /** Getting partition offset from Zookeeper. */
152- private def getCommitOffset (groupId : String , topic : String , partition : Int ): Long = {
151+ private def getCommitOffset (groupId : String , topic : String , partition : Int ): Option [ Long ] = {
153152 assert(zkClient != null , " Zookeeper client is not initialized" )
154-
155153 val topicDirs = new ZKGroupTopicDirs (groupId, topic)
156154 val zkPath = s " ${topicDirs.consumerOffsetDir}/ $partition"
157-
158- ZkUtils .readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong).getOrElse( 0L )
155+ val offset = ZkUtils .readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong)
156+ offset
159157 }
160158}
0 commit comments