Skip to content

Commit b3caf58

Browse files
committed
Fix CheckpointSuite.recovery with file input stream
1 parent 09cb0d9 commit b3caf58

File tree

1 file changed

+10
-8
lines changed

1 file changed

+10
-8
lines changed

streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala

Lines changed: 10 additions & 8 deletions
Original file line numberDiff line numberDiff line change
@@ -424,11 +424,11 @@ class CheckpointSuite extends TestSuiteBase {
424424
}
425425
}
426426
}
427-
clock.advance(batchDuration.milliseconds)
428427
eventually(eventuallyTimeout) {
429428
// Wait until all files have been recorded and all batches have started
430429
assert(recordedFiles(ssc) === Seq(1, 2, 3) && batchCounter.getNumStartedBatches === 3)
431430
}
431+
clock.advance(batchDuration.milliseconds)
432432
// Wait for a checkpoint to be written
433433
eventually(eventuallyTimeout) {
434434
assert(Checkpoint.getCheckpointFiles(checkpointDir).size === 6)
@@ -454,9 +454,12 @@ class CheckpointSuite extends TestSuiteBase {
454454
// recorded before failure were saved and successfully recovered
455455
logInfo("*********** RESTARTING ************")
456456
withStreamingContext(new StreamingContext(checkpointDir)) { ssc =>
457-
// So that the restarted StreamingContext's clock has gone forward in time since failure
458-
ssc.conf.set("spark.streaming.manualClock.jump", (batchDuration * 3).milliseconds.toString)
459-
val oldClockTime = clock.getTimeMillis()
457+
// "batchDuration.milliseconds * 3" has gone before restarting StreamingContext. And because
458+
// the recovery time is read from the checkpoint time but the original clock doesn't align
459+
// with the batch time, we need to add the offset "batchDuration.milliseconds / 2".
460+
ssc.conf.set("spark.streaming.manualClock.jump",
461+
(batchDuration.milliseconds / 2 + batchDuration.milliseconds * 3).toString)
462+
val oldClockTime = clock.getTimeMillis() // 15000ms
460463
clock = ssc.scheduler.clock.asInstanceOf[ManualClock]
461464
val batchCounter = new BatchCounter(ssc)
462465
val outputStream = ssc.graph.getOutputStreams().head.asInstanceOf[TestOutputStream[Int]]
@@ -467,10 +470,10 @@ class CheckpointSuite extends TestSuiteBase {
467470
ssc.start()
468471
// Verify that the clock has traveled forward to the expected time
469472
eventually(eventuallyTimeout) {
470-
clock.getTimeMillis() === oldClockTime
473+
assert(clock.getTimeMillis() === oldClockTime)
471474
}
472-
// Wait for pre-failure batch to be recomputed (3 while SSC was down plus last batch)
473-
val numBatchesAfterRestart = 4
475+
// There are 5 batches between 6000ms and 15000ms (inclusive).
476+
val numBatchesAfterRestart = 5
474477
eventually(eventuallyTimeout) {
475478
assert(batchCounter.getNumCompletedBatches === numBatchesAfterRestart)
476479
}
@@ -483,7 +486,6 @@ class CheckpointSuite extends TestSuiteBase {
483486
assert(batchCounter.getNumCompletedBatches === index + numBatchesAfterRestart + 1)
484487
}
485488
}
486-
clock.advance(batchDuration.milliseconds)
487489
logInfo("Output after restart = " + outputStream.output.mkString("[", ", ", "]"))
488490
assert(outputStream.output.size > 0, "No files processed after restart")
489491
ssc.stop()

0 commit comments

Comments
 (0)