Skip to content

Commit

Permalink
[SPARK-18670][SS] Limit the number of StreamingQueryListener.StreamPr…
Browse files Browse the repository at this point in the history
…ogressEvent when there is no data

## What changes were proposed in this pull request?

This PR adds a sql conf `spark.sql.streaming.noDataReportInterval` to control how long to wait before outputing the next StreamProgressEvent when there is no data.

## How was this patch tested?

The added unit test.

Author: Shixiong Zhu <shixiong@databricks.com>

Closes apache#16108 from zsxwing/SPARK-18670.

(cherry picked from commit 56a503d)
Signed-off-by: Tathagata Das <tathagata.das1565@gmail.com>
  • Loading branch information
zsxwing authored and tdas committed Dec 2, 2016
1 parent f915f81 commit f537632
Show file tree
Hide file tree
Showing 3 changed files with 71 additions and 1 deletion.
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,9 @@ class StreamExecution(

private val pollingDelayMs = sparkSession.sessionState.conf.streamingPollingDelay

private val noDataProgressEventInterval =
sparkSession.sessionState.conf.streamingNoDataProgressEventInterval

/**
* A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation.
*/
Expand Down Expand Up @@ -196,6 +199,9 @@ class StreamExecution(
// While active, repeatedly attempt to run batches.
SparkSession.setActiveSession(sparkSession)

// The timestamp we report an event that has no input data
var lastNoDataProgressEventTime = Long.MinValue

triggerExecutor.execute(() => {
startTrigger()

Expand All @@ -218,7 +224,17 @@ class StreamExecution(

// Report trigger as finished and construct progress object.
finishTrigger(dataAvailable)
postEvent(new QueryProgressEvent(lastProgress))
if (dataAvailable) {
// Reset noDataEventTimestamp if we processed any data
lastNoDataProgressEventTime = Long.MinValue
postEvent(new QueryProgressEvent(lastProgress))
} else {
val now = triggerClock.getTimeMillis()
if (now - noDataProgressEventInterval >= lastNoDataProgressEventTime) {
lastNoDataProgressEventTime = now
postEvent(new QueryProgressEvent(lastProgress))
}
}

if (dataAvailable) {
// We'll increase currentBatchId after we complete processing current batch's data
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -577,6 +577,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")
.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.")
Expand Down Expand Up @@ -658,6 +665,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)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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

Expand All @@ -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") {
Expand Down Expand Up @@ -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 {
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.
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.
Expand Down

0 comments on commit f537632

Please sign in to comment.