Skip to content

Conversation

@zsxwing
Copy link
Member

@zsxwing zsxwing commented Dec 1, 2016

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.

@zsxwing
Copy link
Member Author

zsxwing commented Dec 1, 2016

cc @tdas

.timeConf(TimeUnit.MILLISECONDS)
.createWithDefault(10L)

val STREAMING_NO_DATA_REPORT_INTERVAL =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Report is not a term used anywhere. How about streamingNoDataEventInterval

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")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

... (in ms) ...

Copy link
Member Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

TimeConf supports various units like 10s, 100ms. The unit here is telling which unit we want to use when getting this conf.

if (dataAvailable) {
postEvent(new QueryProgressEvent(lastProgress))
} else {
val now = triggerClock.getTimeMillis()
Copy link
Contributor

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?

withSQLConf(SQLConf.STREAMING_NO_DATA_REPORT_INTERVAL.key -> "100ms") {
@volatile var progressEventCount = 0

val listener = new StreamingQueryListener {
Copy link
Contributor

Choose a reason for hiding this comment

The 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.

assert(queryQueryTerminated.exception === newQueryTerminated.exception)
}

test("noDataReportInterval") {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: make the name better - only one progressevent per interval when no data

Copy link
Contributor

@tdas tdas left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think the test can be simplified.

SparkSession.setActiveSession(sparkSession)

// The timestamp we report an event that has no input data
var noDataEventTimestamp = Long.MinValue
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

oh snap.... i didnt realize... this now more confusing with event time and watermark stuff. sorry! may be rename to lastNoDataProgressEventTime?


private val pollingDelayMs = sparkSession.sessionState.conf.streamingPollingDelay

private val noDataEventInterval = sparkSession.sessionState.conf.streamingNoDataEventInterval
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

similar to below, noDataProgressEventInterval?

.createWithDefault(10L)

val STREAMING_NO_DATA_EVENT_INTERVAL =
SQLConfigBuilder("spark.sql.streaming.noDataEventInterval")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

noDataProgressEventInterval

}
}
testStream(MemoryStream[Int].toDS)(actions: _*)
// 11 is the max value of the possible numbers of events.
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

why 11? explanation?

Copy link
Contributor

@tdas tdas Dec 2, 2016

Choose a reason for hiding this comment

The 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.

val input = new MemoryStream() {
  @volatile var numTriggers = 0
  override def getOffset(): Offset = { numTriggers +=1 ; super.getOffset() } 
}
...
// add listener with progressEventCount and flag isTerminationEventPosted
...
testStream(input.toDF) {   // on real clock
   AddData(...)
   AssertOnQuery {  q => 
      eventually (timeout(streamingTimeout)) {
        assert(input.numTriggers > 100)   // at least 100 triggers have occurred
      }
   }, 
   StopStream, 
   AssertOnQuery { q => 
     eventually(timeout(streamingTimeout) {
       assert(isTerminationEventPosted)   // event queue has drained
       assert(numProgressEvent >=1 && numProgressEvent <= 100)  // no more than 10 events. 
    } 
  }
}

Copy link
Contributor

@tdas tdas Dec 2, 2016

Choose a reason for hiding this comment

The 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.

actions += AssertOnQuery { _ =>
// Sleep so that if the config `noDataEventInterval` doesn't work, it has enough time
// to report too many events.
Thread.sleep(10)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

this combination of manual clock and real time sleep is quite confusing

@SparkQA
Copy link

SparkQA commented Dec 2, 2016

Test build #69505 has finished for PR 16108 at commit 4ace2bd.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 2, 2016

Test build #69512 has finished for PR 16108 at commit 3425231.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

Copy link
Contributor

@tdas tdas left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Test looks good now. But there are other comments that need to be addressed. The name is still confusing, in a different way now. I should have thought of that before suggesting.

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)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Shouldnt this be > 1? What if there is a bug that it only outputs once when isData goes from true -> false, and never outputs again, irrespective of the noDataInterval?

@SparkQA
Copy link

SparkQA commented Dec 2, 2016

Test build #69526 has finished for PR 16108 at commit f7b8755.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@SparkQA
Copy link

SparkQA commented Dec 2, 2016

Test build #69545 has finished for PR 16108 at commit be3737f.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@tdas
Copy link
Contributor

tdas commented Dec 2, 2016

LGTM. Merging to master and 2.1

@asfgit asfgit closed this in 56a503d Dec 2, 2016
asfgit pushed a commit that referenced this pull request Dec 2, 2016
…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 <[email protected]>

Closes #16108 from zsxwing/SPARK-18670.

(cherry picked from commit 56a503d)
Signed-off-by: Tathagata Das <[email protected]>
@zsxwing zsxwing deleted the SPARK-18670 branch December 2, 2016 21:28
robert3005 pushed a commit to palantir/spark that referenced this pull request Dec 15, 2016
…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 <[email protected]>

Closes apache#16108 from zsxwing/SPARK-18670.
uzadude pushed a commit to uzadude/spark that referenced this pull request Jan 27, 2017
…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 <[email protected]>

Closes apache#16108 from zsxwing/SPARK-18670.
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment

Labels

None yet

Projects

None yet

Development

Successfully merging this pull request may close these issues.

3 participants