Github user tdas commented on a diff in the pull request:
https://github.com/apache/spark/pull/15949#discussion_r89424696
--- Diff:
sql/core/src/test/scala/org/apache/spark/sql/streaming/WatermarkSuite.scala ---
@@ -96,28 +96,58 @@ class WatermarkSuite extends StreamTest with
BeforeAndAfter with Logging {
)
}
- ignore("recovery") {
- val inputData = MemoryStream[Int]
-
- val windowedAggregation = inputData.toDF()
+ test("recovery") {
+ val ms = new MemoryStream[Int](0, sqlContext)
+ val df = ms.toDF().toDF("a")
+ val tableName = "recovery"
+ def startQuery: StreamingQuery = {
+ ms.toDF()
.withColumn("eventTime", $"value".cast("timestamp"))
.withWatermark("eventTime", "10 seconds")
.groupBy(window($"eventTime", "5 seconds") as 'window)
.agg(count("*") as 'count)
.select($"window".getField("start").cast("long").as[Long],
$"count".as[Long])
+ .writeStream
+ .format("memory")
+ .queryName(tableName)
+ .outputMode("append")
+ .start()
+ }
- testStream(windowedAggregation)(
- AddData(inputData, 10, 11, 12, 13, 14, 15),
- CheckAnswer(),
- AddData(inputData, 25), // Advance watermark to 15 seconds
- StopStream,
- StartStream(),
- CheckAnswer(),
- AddData(inputData, 25), // Evict items less than previous watermark.
- StopStream,
- StartStream(),
- CheckAnswer((10, 5))
+ var q = startQuery
+ ms.addData(10, 11, 12, 13, 14, 15)
+ q.processAllAvailable()
+
+ checkAnswer(
+ spark.table(tableName), Seq()
+ )
+
+ // Advance watermark to 15 seconds,
+ // but do not process batch
+ ms.addData(25)
+ q.stop()
--- End diff --
why dont you want to process the batch?
let it process the batch, check whether the results are correct (i.e.
things were evicted, and then stop.
drop the table, and restart, processAllAvailable, and restart whether the
same result is recreated.
this will then actually verify that watermark is recovered, and used to
evict the records again.
---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]