-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18670][SS]Limit the number of StreamingQueryListener.StreamProgressEvent when there is no data #16108
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 1 commit
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 |
|---|---|---|
|
|
@@ -603,6 +603,13 @@ object SQLConf { | |
| .timeConf(TimeUnit.MILLISECONDS) | ||
| .createWithDefault(10L) | ||
|
|
||
| val STREAMING_NO_DATA_REPORT_INTERVAL = | ||
|
||
| SQLConfigBuilder("spark.sql.streaming.noDataReportInterval") | ||
| .internal() | ||
| .doc("How long to wait between two progress events when there is no data") | ||
|
Contributor
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. ... (in ms) ...
Member
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. TimeConf supports various units like |
||
| .timeConf(TimeUnit.MILLISECONDS) | ||
| .createWithDefault(10000L) | ||
|
|
||
| val STREAMING_METRICS_ENABLED = | ||
| SQLConfigBuilder("spark.sql.streaming.metricsEnabled") | ||
| .doc("Whether Dropwizard/Codahale metrics will be reported for active streaming queries.") | ||
|
|
@@ -684,6 +691,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { | |
|
|
||
| def streamingPollingDelay: Long = getConf(STREAMING_POLLING_DELAY) | ||
|
|
||
| def streamingNoDataReportInterval: Long = getConf(STREAMING_NO_DATA_REPORT_INTERVAL) | ||
|
|
||
| def streamingMetricsEnabled: Boolean = getConf(STREAMING_METRICS_ENABLED) | ||
|
|
||
| def streamingProgressRetention: Int = getConf(STREAMING_PROGRESS_RETENTION) | ||
|
|
||
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -31,6 +31,7 @@ import org.scalatest.PrivateMethodTester._ | |
| import org.apache.spark.SparkException | ||
| import org.apache.spark.scheduler._ | ||
| import org.apache.spark.sql.execution.streaming._ | ||
| import org.apache.spark.sql.internal.SQLConf | ||
| import org.apache.spark.sql.streaming.StreamingQueryListener._ | ||
| import org.apache.spark.util.JsonProtocol | ||
|
|
||
|
|
@@ -191,6 +192,48 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { | |
| assert(queryQueryTerminated.exception === newQueryTerminated.exception) | ||
| } | ||
|
|
||
| test("noDataReportInterval") { | ||
|
||
| withSQLConf(SQLConf.STREAMING_NO_DATA_REPORT_INTERVAL.key -> "100ms") { | ||
| @volatile var progressEventCount = 0 | ||
|
|
||
| val listener = new StreamingQueryListener { | ||
|
Contributor
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. can you write a short explanation on what this test does. it hard to understand. |
||
| override def onQueryStarted(event: QueryStartedEvent): Unit = {} | ||
|
|
||
| override def onQueryProgress(event: QueryProgressEvent): Unit = { | ||
| progressEventCount += 1 | ||
| } | ||
|
|
||
| override def onQueryTerminated(event: QueryTerminatedEvent): Unit = {} | ||
| } | ||
| spark.streams.addListener(listener) | ||
| try { | ||
| val clock = new StreamManualClock() | ||
| val actions = mutable.ArrayBuffer[StreamAction]() | ||
| actions += StartStream(trigger = ProcessingTime(10), triggerClock = clock) | ||
| actions += AssertOnQuery { _ => | ||
| // It should report at least one progress | ||
| eventually(timeout(streamingTimeout)) { | ||
| assert(progressEventCount > 0) | ||
| } | ||
| true | ||
| } | ||
| for (_ <- 1 to 100) { | ||
| actions += AdvanceManualClock(10) | ||
| actions += AssertOnQuery { _ => | ||
| // Sleep so that if the config `noDataReportInterval` doesn't work, it has enough time | ||
| // to report too many events. | ||
| Thread.sleep(10) | ||
|
||
| true | ||
| } | ||
| } | ||
| testStream(MemoryStream[Int].toDS)(actions: _*) | ||
| assert(progressEventCount <= 11) | ||
| } finally { | ||
| spark.streams.removeListener(listener) | ||
| } | ||
| } | ||
| } | ||
|
|
||
| testQuietly("ReplayListenerBus should ignore broken event jsons generated in 2.0.0") { | ||
| // query-event-logs-version-2.0.0.txt has all types of events generated by | ||
| // Structured Streaming in Spark 2.0.0. | ||
|
|
||
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.
Shouldnt there be an event every time the system switches from dataAvailable = true, to dataAvailable = false. This logic wont do that right?