17
17
18
18
package org .apache .spark .streaming .kafka
19
19
20
+
20
21
import java .io .File
21
22
22
23
import scala .collection .mutable
23
24
import scala .concurrent .duration ._
24
25
import scala .language .postfixOps
25
26
import scala .util .Random
26
27
28
+ import com .google .common .io .Files
27
29
import kafka .serializer .StringDecoder
28
30
import kafka .utils .{ZKGroupTopicDirs , ZkUtils }
31
+ import org .apache .commons .io .FileUtils
29
32
import org .scalatest .BeforeAndAfter
30
33
import org .scalatest .concurrent .Eventually
31
34
35
+ import org .apache .spark .SparkConf
32
36
import 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 }
35
38
36
39
class 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" )
38
45
val data = Map (" a" -> 10 , " b" -> 10 , " c" -> 10 )
46
+
47
+ var topic : String = _
39
48
var groupId : String = _
40
49
var kafkaParams : Map [String , String ] = _
50
+ var ssc : StreamingContext = _
51
+ var tempDirectory : File = null
41
52
42
53
before {
43
- beforeFunction() // call this first to start ZK and Kafka
54
+ setupKafka()
55
+ topic = s " test-topic- ${Random .nextInt(10000 )}"
44
56
groupId = s " test-consumer- ${Random .nextInt(10000 )}"
45
57
kafkaParams = Map (
46
58
" zookeeper.connect" -> zkAddress,
47
59
" group.id" -> groupId,
48
60
" auto.offset.reset" -> " smallest"
49
61
)
62
+
63
+ ssc = new StreamingContext (sparkConf, Milliseconds (500 ))
64
+ tempDirectory = Files .createTempDir()
65
+ ssc.checkpoint(tempDirectory.getAbsolutePath)
50
66
}
51
67
52
68
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()
54
77
}
55
78
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" ) {
63
81
createTopic(topic)
64
82
produceAndSendMessage(topic, data)
65
83
84
+ // Verify whether the offset of this group/topic/partition is 0 before starting.
85
+ assert(getCommitOffset(groupId, topic, 0 ) === None )
86
+
66
87
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 )
71
89
val result = new mutable.HashMap [String , Long ]()
72
90
stream.map { case (k, v) => v }.foreachRDD { r =>
73
91
val ret = r.collect()
@@ -77,84 +95,64 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter
77
95
}
78
96
}
79
97
ssc.start()
80
- eventually(timeout(10000 milliseconds), interval(100 milliseconds)) {
98
+
99
+ eventually(timeout(20000 milliseconds), interval(200 milliseconds)) {
81
100
// A basic process verification for ReliableKafkaReceiver.
82
101
// Verify whether received message number is equal to the sent message number.
83
102
assert(data.size === result.size)
84
103
// Verify whether each message is the same as the data to be verified.
85
104
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
+
86
108
}
87
109
ssc.stop()
88
110
}
111
+ /*
89
112
test("Verify the offset commit") {
90
113
// 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
-
98
114
createTopic(topic)
99
115
produceAndSendMessage(topic, data)
100
116
101
- // Verify whether the offset of this group/topic/partition is 0 before starting.
102
- assert(getCommitOffset(groupId, topic, 0 ) === 0L )
103
-
104
117
// Do this to consume all the message of this group/topic.
105
118
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)
110
120
stream.foreachRDD(_ => Unit)
111
121
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)) {
115
123
}
116
124
ssc.stop()
117
125
}
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" ) {
127
128
val topics = Map (" topic1" -> 1 , " topic2" -> 1 , " topic3" -> 1 )
128
129
topics.foreach { case (t, _) =>
129
130
createTopic(t)
130
131
produceAndSendMessage(t, data)
131
132
}
132
133
133
134
// 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 ) }
135
136
136
137
// Consuming all the data sent to the broker which will potential commit the offsets internally.
137
138
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 )
142
140
stream.foreachRDD(_ => Unit )
143
141
ssc.start()
144
- eventually(timeout(3000 milliseconds), interval(100 milliseconds)) {
142
+ eventually(timeout(20000 milliseconds), interval(100 milliseconds)) {
145
143
// 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 ) ) }
147
145
}
148
146
ssc.stop()
149
147
}
150
148
149
+
151
150
/** 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 ] = {
153
152
assert(zkClient != null , " Zookeeper client is not initialized" )
154
-
155
153
val topicDirs = new ZKGroupTopicDirs (groupId, topic)
156
154
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
159
157
}
160
158
}
0 commit comments