-
Notifications
You must be signed in to change notification settings - Fork 28k
/
TestSuiteBase.scala
572 lines (510 loc) · 20.8 KB
/
TestSuiteBase.scala
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.spark.streaming
import java.io.{File, IOException, ObjectInputStream}
import java.util.concurrent.ConcurrentLinkedQueue
import scala.collection.JavaConverters._
import scala.language.implicitConversions
import scala.reflect.ClassTag
import org.scalatest.BeforeAndAfter
import org.scalatest.concurrent.Eventually.timeout
import org.scalatest.concurrent.PatienceConfiguration
import org.scalatest.time.{Seconds => ScalaTestSeconds, Span}
import org.apache.spark.{SparkConf, SparkFunSuite}
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.streaming.dstream.{DStream, ForEachDStream, InputDStream}
import org.apache.spark.streaming.scheduler._
import org.apache.spark.util.{ManualClock, Utils}
/**
* A dummy stream that does absolutely nothing.
*/
private[streaming] class DummyDStream(ssc: StreamingContext) extends DStream[Int](ssc) {
override def dependencies: List[DStream[Int]] = List.empty
override def slideDuration: Duration = Seconds(1)
override def compute(time: Time): Option[RDD[Int]] = Some(ssc.sc.emptyRDD[Int])
}
/**
* A dummy input stream that does absolutely nothing.
*/
private[streaming] class DummyInputDStream(ssc: StreamingContext) extends InputDStream[Int](ssc) {
override def start(): Unit = { }
override def stop(): Unit = { }
override def compute(time: Time): Option[RDD[Int]] = Some(ssc.sc.emptyRDD[Int])
}
/**
* This is a input stream just for the testsuites. This is equivalent to a checkpointable,
* replayable, reliable message queue like Kafka. It requires a sequence as input, and
* returns the i_th element at the i_th batch under manual clock.
*/
class TestInputStream[T: ClassTag](_ssc: StreamingContext, input: Seq[Seq[T]], numPartitions: Int)
extends InputDStream[T](_ssc) {
def start() {}
def stop() {}
def compute(validTime: Time): Option[RDD[T]] = {
logInfo("Computing RDD for time " + validTime)
val index = ((validTime - zeroTime) / slideDuration - 1).toInt
val selectedInput = if (index < input.size) input(index) else Seq[T]()
// lets us test cases where RDDs are not created
if (selectedInput == null) {
return None
}
// Report the input data's information to InputInfoTracker for testing
val inputInfo = StreamInputInfo(id, selectedInput.length.toLong)
ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo)
val rdd = ssc.sc.makeRDD(selectedInput, numPartitions)
logInfo("Created RDD " + rdd.id + " with " + selectedInput)
Some(rdd)
}
}
/**
* This is a output stream just for the testsuites. All the output is collected into a
* ConcurrentLinkedQueue. This queue is wiped clean on being restored from checkpoint.
*
* The buffer contains a sequence of RDD's, each containing a sequence of items.
*/
class TestOutputStream[T: ClassTag](
parent: DStream[T],
val output: ConcurrentLinkedQueue[Seq[T]] =
new ConcurrentLinkedQueue[Seq[T]]()
) extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
val collected = rdd.collect()
output.add(collected)
}, false) {
// This is to clear the output buffer every it is read from a checkpoint
@throws(classOf[IOException])
private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException {
ois.defaultReadObject()
output.clear()
}
}
/**
* This is a output stream just for the testsuites. All the output is collected into a
* ConcurrentLinkedQueue. This queue is wiped clean on being restored from checkpoint.
*
* The queue contains a sequence of RDD's, each containing a sequence of partitions, each
* containing a sequence of items.
*/
class TestOutputStreamWithPartitions[T: ClassTag](
parent: DStream[T],
val output: ConcurrentLinkedQueue[Seq[Seq[T]]] =
new ConcurrentLinkedQueue[Seq[Seq[T]]]())
extends ForEachDStream[T](parent, (rdd: RDD[T], t: Time) => {
val collected = rdd.glom().collect().map(_.toSeq)
output.add(collected)
}, false) {
// This is to clear the output buffer every it is read from a checkpoint
@throws(classOf[IOException])
private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException {
ois.defaultReadObject()
output.clear()
}
}
/**
* An object that counts the number of started / completed batches. This is implemented using a
* StreamingListener. Constructing a new instance automatically registers a StreamingListener on
* the given StreamingContext.
*/
class BatchCounter(ssc: StreamingContext) {
// All access to this state should be guarded by `BatchCounter.this.synchronized`
private var numCompletedBatches = 0
private var numStartedBatches = 0
private var lastCompletedBatchTime: Time = null
private val listener = new StreamingListener {
override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit =
BatchCounter.this.synchronized {
numStartedBatches += 1
BatchCounter.this.notifyAll()
}
override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit =
BatchCounter.this.synchronized {
numCompletedBatches += 1
lastCompletedBatchTime = batchCompleted.batchInfo.batchTime
BatchCounter.this.notifyAll()
}
}
ssc.addStreamingListener(listener)
def getNumCompletedBatches: Int = this.synchronized {
numCompletedBatches
}
def getNumStartedBatches: Int = this.synchronized {
numStartedBatches
}
def getLastCompletedBatchTime: Time = this.synchronized {
lastCompletedBatchTime
}
/**
* Wait until `expectedNumCompletedBatches` batches are completed, or timeout. Return true if
* `expectedNumCompletedBatches` batches are completed. Otherwise, return false to indicate it's
* timeout.
*
* @param expectedNumCompletedBatches the `expectedNumCompletedBatches` batches to wait
* @param timeout the maximum time to wait in milliseconds.
*/
def waitUntilBatchesCompleted(expectedNumCompletedBatches: Int, timeout: Long): Boolean =
waitUntilConditionBecomeTrue(numCompletedBatches >= expectedNumCompletedBatches, timeout)
/**
* Wait until `expectedNumStartedBatches` batches are completed, or timeout. Return true if
* `expectedNumStartedBatches` batches are completed. Otherwise, return false to indicate it's
* timeout.
*
* @param expectedNumStartedBatches the `expectedNumStartedBatches` batches to wait
* @param timeout the maximum time to wait in milliseconds.
*/
def waitUntilBatchesStarted(expectedNumStartedBatches: Int, timeout: Long): Boolean =
waitUntilConditionBecomeTrue(numStartedBatches >= expectedNumStartedBatches, timeout)
private def waitUntilConditionBecomeTrue(condition: => Boolean, timeout: Long): Boolean = {
synchronized {
var now = System.currentTimeMillis()
val timeoutTick = now + timeout
while (!condition && timeoutTick > now) {
wait(timeoutTick - now)
now = System.currentTimeMillis()
}
condition
}
}
}
/**
* This is the base trait for Spark Streaming testsuites. This provides basic functionality
* to run user-defined set of input on user-defined stream operations, and verify the output.
*/
trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging {
// Name of the framework for Spark context
def framework: String = this.getClass.getSimpleName
// Master for Spark context
def master: String = "local[2]"
// Batch duration
def batchDuration: Duration = Seconds(1)
// Directory where the checkpoint data will be saved
lazy val checkpointDir: String = {
val dir = Utils.createTempDir()
logDebug(s"checkpointDir: $dir")
dir.toString
}
// Number of partitions of the input parallel collections created for testing
def numInputPartitions: Int = 2
// Maximum time to wait before the test times out
def maxWaitTimeMillis: Int = 10000
// Whether to use manual clock or not
def useManualClock: Boolean = true
// Whether to actually wait in real time before changing manual clock
def actuallyWait: Boolean = false
// A SparkConf to use in tests. Can be modified before calling setupStreams to configure things.
val conf = new SparkConf()
.setMaster(master)
.setAppName(framework)
// Timeout for use in ScalaTest `eventually` blocks
val eventuallyTimeout: PatienceConfiguration.Timeout = timeout(Span(10, ScalaTestSeconds))
// Default before function for any streaming test suite. Override this
// if you want to add your stuff to "before" (i.e., don't call before { } )
def beforeFunction() {
if (useManualClock) {
logInfo("Using manual clock")
conf.set("spark.streaming.clock", "org.apache.spark.util.ManualClock")
} else {
logInfo("Using real clock")
conf.set("spark.streaming.clock", "org.apache.spark.util.SystemClock")
}
}
// Default after function for any streaming test suite. Override this
// if you want to add your stuff to "after" (i.e., don't call after { } )
def afterFunction() {
System.clearProperty("spark.streaming.clock")
}
before(beforeFunction)
after(afterFunction)
/**
* Run a block of code with the given StreamingContext and automatically
* stop the context when the block completes or when an exception is thrown.
*/
def withStreamingContext[R](ssc: StreamingContext)(block: StreamingContext => R): R = {
try {
block(ssc)
} finally {
try {
ssc.stop(stopSparkContext = true)
} catch {
case e: Exception =>
logError("Error stopping StreamingContext", e)
}
}
}
/**
* Run a block of code with the given TestServer and automatically
* stop the server when the block completes or when an exception is thrown.
*/
def withTestServer[R](testServer: TestServer)(block: TestServer => R): R = {
try {
block(testServer)
} finally {
try {
testServer.stop()
} catch {
case e: Exception =>
logError("Error stopping TestServer", e)
}
}
}
/**
* Set up required DStreams to test the DStream operation using the two sequences
* of input collections.
*/
def setupStreams[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
numPartitions: Int = numInputPartitions
): StreamingContext = {
// Create StreamingContext
val ssc = new StreamingContext(conf, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir)
}
// Setup the stream computation
val inputStream = new TestInputStream(ssc, input, numPartitions)
val operatedStream = operation(inputStream)
val outputStream = new TestOutputStreamWithPartitions(operatedStream,
new ConcurrentLinkedQueue[Seq[Seq[V]]])
outputStream.register()
ssc
}
/**
* Set up required DStreams to test the binary operation using the sequence
* of input collections.
*/
def setupStreams[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W]
): StreamingContext = {
// Create StreamingContext
val ssc = new StreamingContext(conf, batchDuration)
if (checkpointDir != null) {
ssc.checkpoint(checkpointDir)
}
// Setup the stream computation
val inputStream1 = new TestInputStream(ssc, input1, numInputPartitions)
val inputStream2 = new TestInputStream(ssc, input2, numInputPartitions)
val operatedStream = operation(inputStream1, inputStream2)
val outputStream = new TestOutputStreamWithPartitions(operatedStream,
new ConcurrentLinkedQueue[Seq[Seq[W]]])
outputStream.register()
ssc
}
/**
* Runs the streams set up in `ssc` on manual clock for `numBatches` batches and
* returns the collected output. It will wait until `numExpectedOutput` number of
* output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached.
*
* Returns a sequence of items for each RDD.
*
* @param ssc The StreamingContext
* @param numBatches The number of batches should be run
* @param numExpectedOutput The number of expected output
* @param preStop The function to run before stopping StreamingContext
*/
def runStreams[V: ClassTag](
ssc: StreamingContext,
numBatches: Int,
numExpectedOutput: Int,
preStop: () => Unit = () => {}
): Seq[Seq[V]] = {
// Flatten each RDD into a single Seq
runStreamsWithPartitions(ssc, numBatches, numExpectedOutput, preStop).map(_.flatten.toSeq)
}
/**
* Runs the streams set up in `ssc` on manual clock for `numBatches` batches and
* returns the collected output. It will wait until `numExpectedOutput` number of
* output data has been collected or timeout (set by `maxWaitTimeMillis`) is reached.
*
* Returns a sequence of RDD's. Each RDD is represented as several sequences of items, each
* representing one partition.
*
* @param ssc The StreamingContext
* @param numBatches The number of batches should be run
* @param numExpectedOutput The number of expected output
* @param preStop The function to run before stopping StreamingContext
*/
def runStreamsWithPartitions[V: ClassTag](
ssc: StreamingContext,
numBatches: Int,
numExpectedOutput: Int,
preStop: () => Unit = () => {}
): Seq[Seq[Seq[V]]] = {
assert(numBatches > 0, "Number of batches to run stream computation is zero")
assert(numExpectedOutput > 0, "Number of expected outputs after " + numBatches + " is zero")
logInfo("numBatches = " + numBatches + ", numExpectedOutput = " + numExpectedOutput)
// Get the output buffer
val outputStream = ssc.graph.getOutputStreams.
filter(_.isInstanceOf[TestOutputStreamWithPartitions[_]]).
head.asInstanceOf[TestOutputStreamWithPartitions[V]]
val output = outputStream.output
try {
// Start computation
ssc.start()
// Advance manual clock
val clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
logInfo("Manual clock before advancing = " + clock.getTimeMillis())
if (actuallyWait) {
for (i <- 1 to numBatches) {
logInfo("Actually waiting for " + batchDuration)
clock.advance(batchDuration.milliseconds)
Thread.sleep(batchDuration.milliseconds)
}
} else {
clock.advance(numBatches * batchDuration.milliseconds)
}
logInfo("Manual clock after advancing = " + clock.getTimeMillis())
// Wait until expected number of output items have been generated
val startTime = System.currentTimeMillis()
while (output.size < numExpectedOutput &&
System.currentTimeMillis() - startTime < maxWaitTimeMillis) {
logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput)
ssc.awaitTerminationOrTimeout(50)
}
val timeTaken = System.currentTimeMillis() - startTime
logInfo("Output generated in " + timeTaken + " milliseconds")
output.asScala.foreach(x => logInfo("[" + x.mkString(",") + "]"))
assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms")
assert(output.size === numExpectedOutput, "Unexpected number of outputs generated")
Thread.sleep(100) // Give some time for the forgetting old RDDs to complete
preStop()
} finally {
ssc.stop(stopSparkContext = true)
}
output.asScala.toSeq
}
/**
* Verify whether the output values after running a DStream operation
* is same as the expected output values, by comparing the output
* collections either as lists (order matters) or sets (order does not matter)
*/
def verifyOutput[V: ClassTag](
output: Seq[Seq[V]],
expectedOutput: Seq[Seq[V]],
useSet: Boolean
) {
logInfo("--------------------------------")
logInfo("output.size = " + output.size)
logInfo("output")
output.foreach(x => logInfo("[" + x.mkString(",") + "]"))
logInfo("expected output.size = " + expectedOutput.size)
logInfo("expected output")
expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]"))
logInfo("--------------------------------")
// Match the output with the expected output
for (i <- 0 until output.size) {
if (useSet) {
assert(
output(i).toSet === expectedOutput(i).toSet,
s"Set comparison failed\n" +
s"Expected output (${expectedOutput.size} items):\n${expectedOutput.mkString("\n")}\n" +
s"Generated output (${output.size} items): ${output.mkString("\n")}"
)
} else {
assert(
output(i).toList === expectedOutput(i).toList,
s"Ordered list comparison failed\n" +
s"Expected output (${expectedOutput.size} items):\n${expectedOutput.mkString("\n")}\n" +
s"Generated output (${output.size} items): ${output.mkString("\n")}"
)
}
}
logInfo("Output verified successfully")
}
/**
* Test unary DStream operation with a list of inputs, with number of
* batches to run same as the number of expected output values
*/
def testOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
useSet: Boolean = false
) {
testOperation[U, V](input, operation, expectedOutput, -1, useSet)
}
/**
* Test unary DStream operation with a list of inputs
* @param input Sequence of input collections
* @param operation Binary DStream operation to be applied to the 2 inputs
* @param expectedOutput Sequence of expected output collections
* @param numBatches Number of batches to run the operation for
* @param useSet Compare the output values with the expected output values
* as sets (order matters) or as lists (order does not matter)
*/
def testOperation[U: ClassTag, V: ClassTag](
input: Seq[Seq[U]],
operation: DStream[U] => DStream[V],
expectedOutput: Seq[Seq[V]],
numBatches: Int,
useSet: Boolean
) {
val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
withStreamingContext(setupStreams[U, V](input, operation)) { ssc =>
val output = runStreams[V](ssc, numBatches_, expectedOutput.size)
verifyOutput[V](output.toSeq, expectedOutput, useSet)
}
}
/**
* Test binary DStream operation with two lists of inputs, with number of
* batches to run same as the number of expected output values
*/
def testOperation[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W],
expectedOutput: Seq[Seq[W]],
useSet: Boolean
) {
testOperation[U, V, W](input1, input2, operation, expectedOutput, -1, useSet)
}
/**
* Test binary DStream operation with two lists of inputs
* @param input1 First sequence of input collections
* @param input2 Second sequence of input collections
* @param operation Binary DStream operation to be applied to the 2 inputs
* @param expectedOutput Sequence of expected output collections
* @param numBatches Number of batches to run the operation for
* @param useSet Compare the output values with the expected output values
* as sets (order matters) or as lists (order does not matter)
*/
def testOperation[U: ClassTag, V: ClassTag, W: ClassTag](
input1: Seq[Seq[U]],
input2: Seq[Seq[V]],
operation: (DStream[U], DStream[V]) => DStream[W],
expectedOutput: Seq[Seq[W]],
numBatches: Int,
useSet: Boolean
) {
val numBatches_ = if (numBatches > 0) numBatches else expectedOutput.size
withStreamingContext(setupStreams[U, V, W](input1, input2, operation)) { ssc =>
val output = runStreams[W](ssc, numBatches_, expectedOutput.size)
verifyOutput[W](output.toSeq, expectedOutput, useSet)
}
}
/**
* Creates a temporary directory, which is then passed to `f` and will be deleted after `f`
* returns.
* (originally from `SqlTestUtils`.)
* @todo Probably this method should be moved to a more general place
*/
protected def withTempDir(f: File => Unit): Unit = {
val dir = Utils.createTempDir().getCanonicalFile
try f(dir) finally Utils.deleteRecursively(dir)
}
}