Skip to content

Commit be3737f

Browse files
committed
Address more
1 parent f7b8755 commit be3737f

3 files changed

Lines changed: 12 additions & 10 deletions

File tree

sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala

Lines changed: 6 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -63,7 +63,8 @@ class StreamExecution(
6363

6464
private val pollingDelayMs = sparkSession.sessionState.conf.streamingPollingDelay
6565

66-
private val noDataEventInterval = sparkSession.sessionState.conf.streamingNoDataEventInterval
66+
private val noDataProgressEventInterval =
67+
sparkSession.sessionState.conf.streamingNoDataProgressEventInterval
6768

6869
/**
6970
* A lock used to wait/notify when batches complete. Use a fair lock to avoid thread starvation.
@@ -199,7 +200,7 @@ class StreamExecution(
199200
SparkSession.setActiveSession(sparkSession)
200201

201202
// The timestamp we report an event that has no input data
202-
var noDataEventTimestamp = Long.MinValue
203+
var lastNoDataProgressEventTime = Long.MinValue
203204

204205
triggerExecutor.execute(() => {
205206
startTrigger()
@@ -225,12 +226,12 @@ class StreamExecution(
225226
finishTrigger(dataAvailable)
226227
if (dataAvailable) {
227228
// Reset noDataEventTimestamp if we processed any data
228-
noDataEventTimestamp = Long.MinValue
229+
lastNoDataProgressEventTime = Long.MinValue
229230
postEvent(new QueryProgressEvent(lastProgress))
230231
} else {
231232
val now = triggerClock.getTimeMillis()
232-
if (now - noDataEventInterval >= noDataEventTimestamp) {
233-
noDataEventTimestamp = now
233+
if (now - noDataProgressEventInterval >= lastNoDataProgressEventTime) {
234+
lastNoDataProgressEventTime = now
234235
postEvent(new QueryProgressEvent(lastProgress))
235236
}
236237
}

sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala

Lines changed: 4 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -603,8 +603,8 @@ object SQLConf {
603603
.timeConf(TimeUnit.MILLISECONDS)
604604
.createWithDefault(10L)
605605

606-
val STREAMING_NO_DATA_EVENT_INTERVAL =
607-
SQLConfigBuilder("spark.sql.streaming.noDataEventInterval")
606+
val STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL =
607+
SQLConfigBuilder("spark.sql.streaming.noDataProgressEventInterval")
608608
.internal()
609609
.doc("How long to wait between two progress events when there is no data")
610610
.timeConf(TimeUnit.MILLISECONDS)
@@ -691,7 +691,8 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging {
691691

692692
def streamingPollingDelay: Long = getConf(STREAMING_POLLING_DELAY)
693693

694-
def streamingNoDataEventInterval: Long = getConf(STREAMING_NO_DATA_EVENT_INTERVAL)
694+
def streamingNoDataProgressEventInterval: Long =
695+
getConf(STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL)
695696

696697
def streamingMetricsEnabled: Boolean = getConf(STREAMING_METRICS_ENABLED)
697698

sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -195,7 +195,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
195195

196196
test("only one progress event per interval when no data") {
197197
// This test will start a query but not push any data, and then check if we push too many events
198-
withSQLConf(SQLConf.STREAMING_NO_DATA_EVENT_INTERVAL.key -> "100ms") {
198+
withSQLConf(SQLConf.STREAMING_NO_DATA_PROGRESS_EVENT_INTERVAL.key -> "100ms") {
199199
@volatile var numProgressEvent = 0
200200
val listener = new StreamingQueryListener {
201201
override def onQueryStarted(event: QueryStartedEvent): Unit = {}
@@ -228,7 +228,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter {
228228
testStream(input.toDS)(actions: _*)
229229
spark.sparkContext.listenerBus.waitUntilEmpty(10000)
230230
// 11 is the max value of the possible numbers of events.
231-
assert(numProgressEvent >= 1 && numProgressEvent <= 11)
231+
assert(numProgressEvent > 1 && numProgressEvent <= 11)
232232
} finally {
233233
spark.streams.removeListener(listener)
234234
}

0 commit comments

Comments
 (0)