@@ -35,24 +35,28 @@ import org.apache.spark.streaming.util.ManualClock
35
35
import org .apache .spark .streaming .{TestSuiteBase , TestOutputStream , StreamingContext }
36
36
import org .apache .spark .streaming .flume .sink ._
37
37
38
- class FlumePollingStreamSuite extends TestSuiteBase {
38
+ class FlumePollingStreamSuite extends TestSuiteBase {
39
39
40
40
val testPort = 9999
41
+ val batchCount = 5
42
+ val eventsPerBatch = 100
43
+ val totalEventsPerChannel = batchCount * eventsPerBatch
44
+ val channelCapacity = 5000
41
45
42
46
test(" flume polling test" ) {
43
47
// Set up the streaming context and input streams
44
48
val ssc = new StreamingContext (conf, batchDuration)
45
49
val flumeStream : ReceiverInputDStream [SparkFlumeEvent ] =
46
50
FlumeUtils .createPollingStream(ssc, Seq (new InetSocketAddress (" localhost" , testPort)),
47
- StorageLevel .MEMORY_AND_DISK , 100 , 1 )
51
+ StorageLevel .MEMORY_AND_DISK , eventsPerBatch , 1 )
48
52
val outputBuffer = new ArrayBuffer [Seq [SparkFlumeEvent ]]
49
53
with SynchronizedBuffer [Seq [SparkFlumeEvent ]]
50
54
val outputStream = new TestOutputStream (flumeStream, outputBuffer)
51
55
outputStream.register()
52
56
53
57
// Start the channel and sink.
54
58
val context = new Context ()
55
- context.put(" capacity" , " 5000 " )
59
+ context.put(" capacity" , channelCapacity.toString )
56
60
context.put(" transactionCapacity" , " 1000" )
57
61
context.put(" keep-alive" , " 0" )
58
62
val channel = new MemoryChannel ()
@@ -77,15 +81,16 @@ import org.apache.spark.streaming.flume.sink._
77
81
val ssc = new StreamingContext (conf, batchDuration)
78
82
val addresses = Seq (testPort, testPort + 1 ).map(new InetSocketAddress (" localhost" , _))
79
83
val flumeStream : ReceiverInputDStream [SparkFlumeEvent ] =
80
- FlumeUtils .createPollingStream(ssc, addresses, StorageLevel .MEMORY_AND_DISK , 100 , 5 )
84
+ FlumeUtils .createPollingStream(ssc, addresses, StorageLevel .MEMORY_AND_DISK ,
85
+ eventsPerBatch, 5 )
81
86
val outputBuffer = new ArrayBuffer [Seq [SparkFlumeEvent ]]
82
87
with SynchronizedBuffer [Seq [SparkFlumeEvent ]]
83
88
val outputStream = new TestOutputStream (flumeStream, outputBuffer)
84
89
outputStream.register()
85
90
86
91
// Start the channel and sink.
87
92
val context = new Context ()
88
- context.put(" capacity" , " 5000 " )
93
+ context.put(" capacity" , channelCapacity.toString )
89
94
context.put(" transactionCapacity" , " 1000" )
90
95
context.put(" keep-alive" , " 0" )
91
96
val channel = new MemoryChannel ()
@@ -127,7 +132,7 @@ import org.apache.spark.streaming.flume.sink._
127
132
executorCompletion.take()
128
133
}
129
134
val startTime = System .currentTimeMillis()
130
- while (outputBuffer.size < 5 &&
135
+ while (outputBuffer.size < batchCount * channels.size &&
131
136
System .currentTimeMillis() - startTime < 15000 ) {
132
137
logInfo(" output.size = " + outputBuffer.size)
133
138
Thread .sleep(100 )
@@ -138,9 +143,9 @@ import org.apache.spark.streaming.flume.sink._
138
143
ssc.stop()
139
144
140
145
val flattenedBuffer = outputBuffer.flatten
141
- assert(flattenedBuffer.size === 25 * channels.size)
146
+ assert(flattenedBuffer.size === totalEventsPerChannel * channels.size)
142
147
var counter = 0
143
- for (k <- 0 until channels.size; i <- 0 until 25 ) {
148
+ for (k <- 0 until channels.size; i <- 0 until totalEventsPerChannel ) {
144
149
val eventToVerify = EventBuilder .withBody((channels(k).getName + " - " +
145
150
String .valueOf(i)).getBytes(" utf-8" ),
146
151
Map [String , String ](" test-" + i.toString -> " header" ))
@@ -157,7 +162,7 @@ import org.apache.spark.streaming.flume.sink._
157
162
j += 1
158
163
}
159
164
}
160
- assert(counter === 25 * channels.size)
165
+ assert(counter === totalEventsPerChannel * channels.size)
161
166
}
162
167
163
168
def assertChannelIsEmpty (channel : MemoryChannel ) = {
@@ -170,10 +175,10 @@ import org.apache.spark.streaming.flume.sink._
170
175
private class TxnSubmitter (channel : MemoryChannel , clock : ManualClock ) extends Callable [Void ] {
171
176
override def call (): Void = {
172
177
var t = 0
173
- for (i <- 0 until 5 ) {
178
+ for (i <- 0 until batchCount ) {
174
179
val tx = channel.getTransaction
175
180
tx.begin()
176
- for (j <- 0 until 5 ) {
181
+ for (j <- 0 until eventsPerBatch ) {
177
182
channel.put(EventBuilder .withBody((channel.getName + " - " + String .valueOf(t)).getBytes(
178
183
" utf-8" ),
179
184
Map [String , String ](" test-" + t.toString -> " header" )))
0 commit comments