-
Notifications
You must be signed in to change notification settings - Fork 28.5k
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
[SPARK-18670][SS]Limit the number of StreamingQueryListener.StreamProgressEvent when there is no data #16108
Changes from all commits
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_PROGRESS_EVENT_INTERVAL = | ||
SQLConfigBuilder("spark.sql.streaming.noDataProgressEventInterval") | ||
.internal() | ||
.doc("How long to wait between two progress events when there is no data") | ||
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) ... 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,9 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging { | |
|
||
def streamingPollingDelay: Long = getConf(STREAMING_POLLING_DELAY) | ||
|
||
def streamingNoDataProgressEventInterval: Long = | ||
getConf(STREAMING_NO_DATA_PROGRESS_EVENT_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 | ||
|
||
|
@@ -46,6 +47,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { | |
assert(spark.streams.active.isEmpty) | ||
assert(addedListeners.isEmpty) | ||
// Make sure we don't leak any events to the next test | ||
spark.sparkContext.listenerBus.waitUntilEmpty(10000) | ||
} | ||
|
||
testQuietly("single listener, check trigger events are generated correctly") { | ||
|
@@ -191,6 +193,48 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { | |
assert(queryQueryTerminated.exception === newQueryTerminated.exception) | ||
} | ||
|
||
test("only one progress event per interval when no data") { | ||
// This test will start a query but not push any data, and then check if we push too many events | ||
withSQLConf(SQLConf.STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL.key -> "100ms") { | ||
@volatile var numProgressEvent = 0 | ||
val listener = new StreamingQueryListener { | ||
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 = { | ||
numProgressEvent += 1 | ||
} | ||
override def onQueryTerminated(event: QueryTerminatedEvent): Unit = {} | ||
} | ||
spark.streams.addListener(listener) | ||
try { | ||
val input = new MemoryStream[Int](0, sqlContext) { | ||
@volatile var numTriggers = 0 | ||
override def getOffset: Option[Offset] = { | ||
numTriggers += 1 | ||
super.getOffset | ||
} | ||
} | ||
val clock = new StreamManualClock() | ||
val actions = mutable.ArrayBuffer[StreamAction]() | ||
actions += StartStream(trigger = ProcessingTime(10), triggerClock = clock) | ||
for (_ <- 1 to 100) { | ||
actions += AdvanceManualClock(10) | ||
} | ||
actions += AssertOnQuery { _ => | ||
eventually(timeout(streamingTimeout)) { | ||
assert(input.numTriggers > 100) // at least 100 triggers have occurred | ||
} | ||
true | ||
} | ||
testStream(input.toDS)(actions: _*) | ||
spark.sparkContext.listenerBus.waitUntilEmpty(10000) | ||
// 11 is the max value of the possible numbers of events. | ||
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. why 11? explanation? 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 think this test looks a little complicated. Also you are assuming the fact the when there is no data, it will sleep for 10 ms real time. why not something like this.
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. Okay, that wont work as well, because there is not guarantee how long the 100 triggers would take. It may take 100 seconds in a slow machine, in which numProgress events could legitimately have 100 events. Nonetheless manual clock tests can be improved. |
||
assert(numProgressEvent > 1 && numProgressEvent <= 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?