You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2021/09/01 09:29:18 UTC

[GitHub] [spark] HeartSaVioR commented on a change in pull request #33763: [SPARK-36533][SS] Trigger.AvailableNow for running streaming queries like Trigger.Once in multiple batches

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



##########
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 {

Review comment:
       I'm not sure we could do it easily, as we can't let the query be "suspended" after figuring out source offsets to process. The streaming query is running concurrently with the main thread.

##########
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 {

Review comment:
       `assert(q.recentProgress.count(_.numInputRows != 0) == 3)`
   
   This verifies that the query runs three micro-batches instead of one.
   
   Btw, the test `SPARK-36533: Trigger.AvailableNow - checkpointing` covers everything in here now. The changes on `index` determines the number of micro-batch being executed, and we also checked the output DataFrame. I think we can simply remove this test.

##########
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 {

Review comment:
       `assert(q.recentProgress.count(_.numInputRows != 0) == 3)`
   
   This verifies that the query runs three micro-batches instead of one.
   
   Btw, the test `SPARK-36533: Trigger.AvailableNow - checkpointing` covers everything in here now. The change on `index` determines the number of micro-batch being executed, and we also checked the output DataFrame. I think we can simply remove this test.

##########
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org