HeartSaVioR commented on a change in pull request #33763:
URL: https://github.com/apache/spark/pull/33763#discussion_r699727889



##########
File path: 
sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala
##########
@@ -1304,6 +1304,140 @@ class FileStreamSourceSuite extends 
FileStreamSourceTest {
     }
   }
 
+  test("SPARK-36533: Trigger.AvailableNow") {
+    withTempDirs { (src, target) =>
+      val checkpoint = new File(target, "chk").getCanonicalPath
+      val targetDir = new File(target, "data").getCanonicalPath
+      var lastFileModTime: Option[Long] = None
+
+      /** Create a text file with a single data item */
+      def createFile(data: Int): File = {
+        val file = stringToFile(new File(src, s"$data.txt"), data.toString)
+        if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get 
+ 1000)
+        lastFileModTime = Some(file.lastModified)
+        file
+      }
+
+      createFile(1)
+      createFile(2)
+      createFile(3)
+
+      // Set up a query to read text files one at a time
+      val df = spark
+        .readStream
+        .option("maxFilesPerTrigger", 1)
+        .text(src.getCanonicalPath)
+
+      def startQuery(): StreamingQuery = {
+        df.writeStream
+          .format("parquet")
+          .trigger(Trigger.AvailableNow)
+          .option("checkpointLocation", checkpoint)
+          .start(targetDir)
+      }
+      val q = startQuery()
+
+      try {
+        assert(q.awaitTermination(streamingTimeout.toMillis))
+        assert(q.recentProgress.count(_.numInputRows != 0) == 3)
+        checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 
3).map(_.toString).toDF)
+      } finally {
+        q.stop()
+      }
+
+      createFile(4)
+      createFile(5)
+
+      // run a second query
+      val q2 = startQuery()
+      try {
+        assert(q2.awaitTermination(streamingTimeout.toMillis))
+        assert(q2.recentProgress.count(_.numInputRows != 0) == 2)
+        checkAnswer(sql(s"SELECT * from parquet.`$targetDir`"), (1 to 
5).map(_.toString).toDF)
+      } finally {
+        q2.stop()
+      }
+    }
+  }
+
+  test("SPARK-36533: Trigger.AvailableNow - checkpointing") {
+    withTempDirs { (src, target) =>
+      val checkpoint = new File(target, "chk").getCanonicalPath
+      var lastFileModTime: Option[Long] = None
+
+      /** Create a text file with a single data item */
+      def createFile(data: Int): File = {
+        val file = stringToFile(new File(src, s"$data.txt"), data.toString)
+        if (lastFileModTime.nonEmpty) file.setLastModified(lastFileModTime.get 
+ 1000)
+        lastFileModTime = Some(file.lastModified)
+        file
+      }
+
+      createFile(1)
+      createFile(2)
+      createFile(3)
+
+      // Set up a query to read text files one at a time
+      val df = spark
+        .readStream
+        .option("maxFilesPerTrigger", 1)
+        .text(src.getCanonicalPath)
+
+      def startTriggerOnceQuery(): StreamingQuery = {
+        df.writeStream
+          .foreachBatch((_: Dataset[Row], _: Long) => {})
+          .trigger(Trigger.Once)
+          .option("checkpointLocation", checkpoint)
+          .start()
+      }
+
+      // run a query with Trigger.Once first
+      val q = startTriggerOnceQuery()
+
+      try {
+        assert(q.awaitTermination(streamingTimeout.toMillis))
+      } finally {
+        q.stop()
+      }
+
+      var index = 3  // We have processed the first 3 rows in the first query

Review comment:
       nit: probably better to add the comment that it tracks the number of 
micro-batch execution starting from here. The code is intuitive but worth 
having elaboration given the importance of this variable.




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]



---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to