jerrypeng commented on code in PR #38517:
URL: https://github.com/apache/spark/pull/38517#discussion_r1049032417


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/AsyncProgressTrackingMicroBatchExecutionSuite.scala:
##########
@@ -0,0 +1,1865 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming
+
+import java.io.{File, OutputStream}
+import java.util.concurrent.{CountDownLatch, Semaphore, ThreadPoolExecutor, 
TimeUnit}
+
+import scala.collection.mutable.ListBuffer
+
+import org.apache.hadoop.fs.Path
+import org.scalatest.BeforeAndAfter
+import org.scalatest.matchers.should.Matchers
+import org.scalatest.time.{Seconds, Span}
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.streaming.WriteToStream
+import org.apache.spark.sql.connector.read.streaming
+import 
org.apache.spark.sql.execution.streaming.AsyncProgressTrackingMicroBatchExecution.{ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS,
 ASYNC_PROGRESS_TRACKING_ENABLED, 
ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK}
+import org.apache.spark.sql.functions.{column, window}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.streaming.{StreamingQuery, 
StreamingQueryException, StreamTest, Trigger}
+import org.apache.spark.sql.streaming.util.StreamManualClock
+import org.apache.spark.util.{Clock, Utils}
+
+class AsyncProgressTrackingMicroBatchExecutionSuite
+    extends StreamTest
+    with BeforeAndAfter
+    with Matchers {
+
+  import testImplicits._
+
+  after {
+    sqlContext.streams.active.foreach(_.stop())
+  }
+
+  def getListOfFiles(dir: String): List[File] = {
+    val d = new File(dir)
+    if (d.exists && d.isDirectory) {
+      d.listFiles.filter(_.isFile).toList
+    } else {
+      List[File]()
+    }
+  }
+
+  def waitPendingOffsetWrites(streamExecution: StreamExecution): Unit = {
+    
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+    eventually(timeout(Span(5, Seconds))) {
+      streamExecution
+        .asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+        .areWritesPendingOrInProgress() should be(false)
+    }
+  }
+
+  def waitPendingPurges(streamExecution: StreamExecution): Unit = {
+    
assert(streamExecution.isInstanceOf[AsyncProgressTrackingMicroBatchExecution])
+    eventually(timeout(Span(5, Seconds))) {
+      streamExecution
+        .asInstanceOf[AsyncProgressTrackingMicroBatchExecution]
+        .arePendingAsyncPurge should be(false)
+    }
+  }
+
+  // test the basic functionality i.e. happy path
+  test("async WAL commits happy path") {
+    val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+    val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+    val ds = inputData.toDF()
+
+    val tableName = "test"
+
+    def startQuery(): StreamingQuery = {
+      ds.writeStream
+        .format("memory")
+        .queryName(tableName)
+        .option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+        .option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 0)
+        .option("checkpointLocation", checkpointLocation)
+        .start()
+    }
+    val query = startQuery()
+    val expected = new ListBuffer[Row]()
+    for (j <- 0 until 100) {
+      for (i <- 0 until 10) {
+        val v = i + (j * 10)
+        inputData.addData({ v })
+        expected += Row(v)
+      }
+      query.processAllAvailable()
+    }
+
+    checkAnswer(
+      spark.table(tableName),
+      expected.toSeq
+    )
+  }
+
+  test("async WAL commits recovery") {
+    val checkpointLocation = Utils.createTempDir(namePrefix = 
"streaming.metadata").getCanonicalPath
+
+    val inputData = new MemoryStream[Int](id = 0, sqlContext = sqlContext)
+    val ds = inputData.toDF()
+
+    var index = 0
+    // to synchronize producing and consuming messages so that
+    // we can generate and read the desired number of batches
+    var countDownLatch = new CountDownLatch(10)
+    val sem = new Semaphore(1)
+    val data = new ListBuffer[Int]()
+    def startQuery(): StreamingQuery = {
+      ds.writeStream
+        .foreachBatch((ds: Dataset[Row], batchId: Long) => {
+          ds.collect.foreach((row: Row) => {
+            data += row.getInt(0)
+          }: Unit)
+          countDownLatch.countDown()
+          index += 1
+          sem.release()
+        })
+        .option(ASYNC_PROGRESS_TRACKING_ENABLED, true)
+        .option(ASYNC_PROGRESS_TRACKING_CHECKPOINTING_INTERVAL_MS, 0)
+        .option(ASYNC_PROGRESS_TRACKING_OVERRIDE_SINK_SUPPORT_CHECK, true)
+        .option("checkpointLocation", checkpointLocation)
+        .start()
+    }
+    var query = startQuery()
+
+    for (i <- 0 until 10) {
+      sem.acquire()
+      inputData.addData({ i })
+    }
+
+    try {
+      countDownLatch.await(streamingTimeout.toMillis, TimeUnit.MILLISECONDS)
+    } finally {
+      query.stop()
+    }
+
+    assert(index == 10)
+    data should equal(Array(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))

Review Comment:
   The problem is the test will fail:
   
   ListBuffer(0, 1, 2, 3, 4, 5, 6, 7, 8, 9) did not equal 
Array(Range.Inclusive(0, 1, 2, 3, 4, 5, 6, 7, 8, 9))
   



-- 
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-unsubscr...@spark.apache.org

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


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

Reply via email to