-
Notifications
You must be signed in to change notification settings - Fork 29k
[SPARK-18721][SS]Fix ForeachSink with watermark + append #16160
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 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 |
|---|---|---|
|
|
@@ -171,7 +171,7 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf | |
| } | ||
| } | ||
|
|
||
| test("foreach with watermark") { | ||
| test("foreach with watermark: complete") { | ||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val windowedAggregation = inputData.toDF() | ||
|
|
@@ -204,6 +204,72 @@ class ForeachSinkSuite extends StreamTest with SharedSQLContext with BeforeAndAf | |
| query.stop() | ||
| } | ||
| } | ||
|
|
||
| test("foreach with watermark: append") { | ||
|
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. what does this test that is not covered in the previous test "watermark + complete"?
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. @tdas As no eviction with
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. actually complete mode should NOT work when watermark is enabled!! Why does this query still work? Thats material for different PR. So I approve this change in this PR.
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. NVM. watermark is a noop in complete mode. false alarm. |
||
| val inputData = MemoryStream[Int] | ||
|
|
||
| val windowedAggregation = inputData.toDF() | ||
| .withColumn("eventTime", $"value".cast("timestamp")) | ||
| .withWatermark("eventTime", "10 seconds") | ||
| .groupBy(window($"eventTime", "5 seconds") as 'window) | ||
| .agg(count("*") as 'count) | ||
| .select($"count".as[Long]) | ||
| .map(_.toInt) | ||
| .repartition(1) | ||
|
|
||
| val query = windowedAggregation | ||
| .writeStream | ||
| .outputMode(OutputMode.Append) | ||
| .foreach(new TestForeachWriter()) | ||
| .start() | ||
| try { | ||
| inputData.addData(10, 11, 12) | ||
| query.processAllAvailable() | ||
| inputData.addData(25) // Advance watermark to 15 seconds | ||
| query.processAllAvailable() | ||
| inputData.addData(25) // Evict items less than previous watermark | ||
| query.processAllAvailable() | ||
|
|
||
| // There should be 3 batches and only does the last batch contain a value. | ||
| val allEvents = ForeachSinkSuite.allEvents() | ||
| assert(allEvents.size === 3) | ||
| val expectedEvents = Seq( | ||
| Seq( | ||
| ForeachSinkSuite.Open(partition = 0, version = 0), | ||
| ForeachSinkSuite.Close(None) | ||
| ), | ||
| Seq( | ||
| ForeachSinkSuite.Open(partition = 0, version = 1), | ||
| ForeachSinkSuite.Close(None) | ||
| ), | ||
| Seq( | ||
| ForeachSinkSuite.Open(partition = 0, version = 2), | ||
| ForeachSinkSuite.Process(value = 3), | ||
| ForeachSinkSuite.Close(None) | ||
| ) | ||
| ) | ||
| assert(allEvents === expectedEvents) | ||
| } finally { | ||
| query.stop() | ||
| } | ||
| } | ||
|
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. I dont see a test that verifies whether the metrics are correct
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. Added a simple test for metrics |
||
|
|
||
| test("foreach sink should support metrics") { | ||
| val inputData = MemoryStream[Int] | ||
| val query = inputData.toDS() | ||
| .writeStream | ||
| .foreach(new TestForeachWriter()) | ||
| .start() | ||
| try { | ||
| inputData.addData(10, 11, 12) | ||
| query.processAllAvailable() | ||
| val recentProgress = query.recentProgresses.filter(_.numInputRows != 0).headOption | ||
| assert(recentProgress.isDefined && recentProgress.get.numInputRows === 3, | ||
| s"recentProgresses[${query.recentProgresses.toList}] doesn't contain correct metrics") | ||
| } finally { | ||
| query.stop() | ||
| } | ||
| } | ||
| } | ||
|
|
||
| /** A global object to collect events in the executor */ | ||
|
|
||
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.
SPARK-16264was resolved asWon't Fix. So I removed it from the comment.