-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-15022][SPARK-15023][SQL][Streaming] Add support for testing against the ProcessingTime(intervalMS > 0) trigger and ManualClock
#12797
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
Changes from 2 commits
90ed692
9d80b15
b63653a
bc89962
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -65,8 +65,22 @@ case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = | |
| s"${intervalMs} milliseconds, but spent ${realElapsedTimeMs} milliseconds") | ||
| } | ||
|
|
||
| /** Return the next multiple of intervalMs */ | ||
| /** Return the next multiple of intervalMs | ||
| * | ||
| * e.g. for intervalMs = 100 | ||
| * nextBatchTime(0) = 100 | ||
| * nextBatchTime(1) = 100 | ||
| * ... | ||
| * nextBatchTime(99) = 100 | ||
| * nextBatchTime(100) = 200 | ||
| * nextBatchTime(101) = 200 | ||
| * ... | ||
| * nextBatchTime(199) = 200 | ||
| * nextBatchTime(200) = 300 | ||
| * | ||
| * Note, this way, we'll get nextBatchTime(nextBatchTime(0)) = 200, rather than = 0 | ||
|
||
| * */ | ||
| def nextBatchTime(now: Long): Long = { | ||
| (now - 1) / intervalMs * intervalMs + intervalMs | ||
| now / intervalMs * intervalMs + intervalMs | ||
|
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. When I wrote this method, I was trying to deal with one case: If a batch takes exactly However, I forgot to handle the case that a batch takes 0ms. How about changing this line to: if (batchElapsedTimeMs == 0) {
clock.waitTillTime(intervalMs)
} else {
clock.waitTillTime(nextBatchTime(batchEndTimeMs))
}
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. @zsxwing thanks for clarifying on this! :-) [1] if (batchElapsedTimeMs == 0 && batchEndTimeMs % intervalMS == 0) {
clock.waitTillTime(batchEndTimeMs + intervalMs)
} else {
clock.waitTillTime(nextBatchTime(batchEndTimeMs))
}For me It seems a little hard to interpret... [2]
This is a good point! I've done some calculations based on your comments, and it seems we would still run the next batch at once when the last job takes exactly prior to this path: after this patch, it's still the same: @zsxwing given the above [1] and [2], maybe we should simply change
Member
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. I see. I think your approach is better. Thanks for your clarifying. |
||
| } | ||
| } | ||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -38,7 +38,7 @@ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder, Ro | |
| import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan | ||
| import org.apache.spark.sql.catalyst.util._ | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.util.Utils | ||
| import org.apache.spark.util.{Clock, ManualClock, Utils} | ||
|
|
||
| /** | ||
| * A framework for implementing tests for streaming queries and sources. | ||
|
|
@@ -142,7 +142,10 @@ trait StreamTest extends QueryTest with Timeouts { | |
| case object StopStream extends StreamAction with StreamMustBeRunning | ||
|
|
||
| /** Starts the stream, resuming if data has already been processed. It must not be running. */ | ||
| case object StartStream extends StreamAction | ||
| case class StartStream(trigger: Trigger = null, triggerClock: Clock = null) extends StreamAction | ||
|
||
|
|
||
| /** Advance the trigger clock's time manually. */ | ||
| case class AdvanceManualClock(timeToAdd: Long) extends StreamAction | ||
|
|
||
| /** Signals that a failure is expected and should not kill the test. */ | ||
| case class ExpectFailure[T <: Throwable : ClassTag]() extends StreamAction { | ||
|
|
@@ -199,8 +202,8 @@ trait StreamTest extends QueryTest with Timeouts { | |
|
|
||
| // If the test doesn't manually start the stream, we do it automatically at the beginning. | ||
| val startedManually = | ||
| actions.takeWhile(!_.isInstanceOf[StreamMustBeRunning]).contains(StartStream) | ||
| val startedTest = if (startedManually) actions else StartStream +: actions | ||
| actions.takeWhile(!_.isInstanceOf[StreamMustBeRunning]).exists(_.isInstanceOf[StartStream]) | ||
| val startedTest = if (startedManually) actions else StartStream() +: actions | ||
|
|
||
| def testActions = actions.zipWithIndex.map { | ||
| case (a, i) => | ||
|
|
@@ -280,19 +283,35 @@ trait StreamTest extends QueryTest with Timeouts { | |
| try { | ||
| startedTest.foreach { action => | ||
| action match { | ||
| case StartStream => | ||
| case StartStream(_trigger, _triggerClock) => | ||
| verify(currentStream == null, "stream already running") | ||
| lastStream = currentStream | ||
| currentStream = | ||
| sqlContext | ||
| .streams | ||
| .startQuery( | ||
| StreamExecution.nextName, | ||
| metadataRoot, | ||
| stream, | ||
| sink, | ||
| outputMode = outputMode) | ||
| .asInstanceOf[StreamExecution] | ||
| if (_trigger != null) { | ||
|
||
| // we pass in explicit trigger and triggerClock | ||
| sqlContext | ||
| .streams | ||
| .startQuery( | ||
| StreamExecution.nextName, | ||
| metadataRoot, | ||
| stream, | ||
| sink, | ||
| trigger = _trigger, | ||
| triggerClock = _triggerClock, | ||
| outputMode = outputMode) | ||
| .asInstanceOf[StreamExecution] | ||
| } else { | ||
| // we left out trigger and triggerClock as their default values | ||
| sqlContext | ||
| .streams | ||
| .startQuery( | ||
| StreamExecution.nextName, | ||
| metadataRoot, | ||
| stream, | ||
| sink, | ||
| outputMode = outputMode) | ||
| .asInstanceOf[StreamExecution] | ||
| } | ||
| currentStream.microBatchThread.setUncaughtExceptionHandler( | ||
| new UncaughtExceptionHandler { | ||
| override def uncaughtException(t: Thread, e: Throwable): Unit = { | ||
|
|
@@ -301,6 +320,13 @@ trait StreamTest extends QueryTest with Timeouts { | |
| } | ||
| }) | ||
|
|
||
| case AdvanceManualClock(timeToAdd) => | ||
| verify(currentStream != null, | ||
| "can not advance manual clock when a stream is not running") | ||
| verify(currentStream.triggerClock.isInstanceOf[ManualClock], | ||
| s"can not advance clock of type ${currentStream.triggerClock.getClass}") | ||
| currentStream.triggerClock.asInstanceOf[ManualClock].advance(timeToAdd) | ||
|
|
||
| case StopStream => | ||
| verify(currentStream != null, "can not stop a stream that is not running") | ||
| try failAfter(streamingTimeout) { | ||
|
|
@@ -470,7 +496,7 @@ trait StreamTest extends QueryTest with Timeouts { | |
| addRandomData() | ||
|
|
||
| case _ => // StartStream | ||
| actions += StartStream | ||
| actions += StartStream() | ||
| running = true | ||
| } | ||
| } else { | ||
|
|
@@ -488,7 +514,7 @@ trait StreamTest extends QueryTest with Timeouts { | |
| } | ||
| } | ||
| } | ||
| if(!running) { actions += StartStream } | ||
| if(!running) { actions += StartStream() } | ||
| addCheck() | ||
| testStream(ds)(actions: _*) | ||
| } | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -21,19 +21,41 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} | |
|
|
||
| import org.apache.spark.SparkFunSuite | ||
| import org.apache.spark.sql.ProcessingTime | ||
| import org.apache.spark.util.ManualClock | ||
| import org.apache.spark.util.{Clock, ManualClock, SystemClock} | ||
|
|
||
| class ProcessingTimeExecutorSuite extends SparkFunSuite { | ||
|
|
||
| test("nextBatchTime") { | ||
| val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(100)) | ||
| assert(processingTimeExecutor.nextBatchTime(0) === 100) | ||
| assert(processingTimeExecutor.nextBatchTime(1) === 100) | ||
| assert(processingTimeExecutor.nextBatchTime(99) === 100) | ||
| assert(processingTimeExecutor.nextBatchTime(100) === 100) | ||
| assert(processingTimeExecutor.nextBatchTime(100) === 200) | ||
| assert(processingTimeExecutor.nextBatchTime(101) === 200) | ||
| assert(processingTimeExecutor.nextBatchTime(150) === 200) | ||
| } | ||
|
|
||
| private def testNextBatchTimeAgainstClock(clock: Clock) { | ||
| val IntervalMS = 100 | ||
|
||
| val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(IntervalMS), clock) | ||
|
|
||
| val ITERATION = 10 | ||
| var nextBatchTime: Long = 0 | ||
| for (it <- 1 to ITERATION) | ||
| nextBatchTime = processingTimeExecutor.nextBatchTime(nextBatchTime) | ||
|
|
||
| // nextBatchTime should be 1000 | ||
| assert(nextBatchTime === IntervalMS * ITERATION) | ||
|
||
| } | ||
|
|
||
| test("nextBatchTime against SystemClock") { | ||
| testNextBatchTimeAgainstClock(new SystemClock) | ||
| } | ||
|
|
||
| test("nextBatchTime against ManualClock") { | ||
|
||
| testNextBatchTimeAgainstClock(new ManualClock) | ||
| } | ||
|
|
||
| private def testBatchTermination(intervalMs: Long): Unit = { | ||
| var batchCounts = 0 | ||
| val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMs)) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -19,10 +19,10 @@ package org.apache.spark.sql.streaming | |
|
|
||
| import org.apache.spark.sql._ | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.functions._ | ||
| import org.apache.spark.sql.sources.StreamSourceProvider | ||
| import org.apache.spark.sql.test.SharedSQLContext | ||
| import org.apache.spark.sql.types.{IntegerType, StructField, StructType} | ||
| import org.apache.spark.util.ManualClock | ||
|
|
||
| class StreamSuite extends StreamTest with SharedSQLContext { | ||
|
|
||
|
|
@@ -34,11 +34,11 @@ class StreamSuite extends StreamTest with SharedSQLContext { | |
|
|
||
| testStream(mapped)( | ||
| AddData(inputData, 1, 2, 3), | ||
| StartStream, | ||
| StartStream(), | ||
| CheckAnswer(2, 3, 4), | ||
| StopStream, | ||
| AddData(inputData, 4, 5, 6), | ||
| StartStream, | ||
| StartStream(), | ||
| CheckAnswer(2, 3, 4, 5, 6, 7)) | ||
| } | ||
|
|
||
|
|
@@ -70,7 +70,7 @@ class StreamSuite extends StreamTest with SharedSQLContext { | |
| CheckAnswer(1, 2, 3, 4, 5, 6), | ||
| StopStream, | ||
| AddData(inputData1, 7), | ||
| StartStream, | ||
| StartStream(), | ||
| AddData(inputData2, 8), | ||
| CheckAnswer(1, 2, 3, 4, 5, 6, 7, 8)) | ||
| } | ||
|
|
@@ -136,6 +136,22 @@ class StreamSuite extends StreamTest with SharedSQLContext { | |
| testStream(ds)() | ||
| } | ||
| } | ||
|
|
||
| // This would fail for now -- error is "Timed out waiting for stream" | ||
| // Root cause is that data generated in batch 0 may not get processed in batch 1 | ||
| // Let's enable this after SPARK-14942: Reduce delay between batch construction and execution | ||
| ignore("minimize delay between batch construction and execution") { | ||
| val inputData = MemoryStream[Int] | ||
| testStream(inputData.toDS())( | ||
| StartStream(ProcessingTime("10 seconds"), new ManualClock), | ||
| /* -- batch 0 ----------------------- */ | ||
| AddData(inputData, 1), | ||
| AddData(inputData, 2), | ||
| AddData(inputData, 3), | ||
| AdvanceManualClock(10 * 1000), // 10 seconds | ||
| /* -- batch 1 ----------------------- */ | ||
| CheckAnswer(1, 2, 3)) | ||
| } | ||
|
Contributor
Author
There was a problem hiding this comment. Choose a reason for hiding this commentThe reason will be displayed to describe this comment to others. Learn more. The above test takes advantage of the new |
||
| } | ||
|
|
||
| /** | ||
|
|
||
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Nit: comment style is off, we use javadoc style