siying commented on code in PR #50559:
URL: https://github.com/apache/spark/pull/50559#discussion_r2045127704


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala:
##########
@@ -414,6 +414,84 @@ class RocksDBCheckpointFailureInjectionSuite extends 
StreamTest
     }
   }
 
+  case class FailureConf2(logType: String, checkpointFormatVersion: String) {
+    override def toString: String = {
+      s"logType = $logType, checkpointFormatVersion = $checkpointFormatVersion"
+    }
+  }
+
+  // tests to validate the behavior after failures when writing to the commit 
and offset logs
+  Seq(
+    FailureConf2("commits", checkpointFormatVersion = "1"),
+    FailureConf2("commits", checkpointFormatVersion = "2"),
+    FailureConf2("offsets", checkpointFormatVersion = "1"),
+    FailureConf2("offsets", checkpointFormatVersion = "2")).foreach { 
failureConf =>
+    test(s"Progress log fails to write $failureConf") {
+      val hadoopConf = new Configuration()
+      hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, 
fileManagerClassName)
+      val rocksdbChangelogCheckpointingConfKey =
+        RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled"
+
+      withTempDirAllowFailureInjection { (checkpointDir, injectionState) =>
+        withSQLConf(
+          rocksdbChangelogCheckpointingConfKey -> "true",
+          SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key -> "2") {
+          val inputData = MemoryStream[Int]
+          val aggregated =
+            inputData.toDF()
+              .groupBy($"value")
+              .agg(count("*"))
+              .as[(Int, Long)]
+
+          // This should cause the second batch to fail
+          injectionState.createAtomicDelayCloseRegex = 
Seq(s".*/${failureConf.logType}/1")
+
+          val additionalConfs = Map(
+            rocksdbChangelogCheckpointingConfKey -> "true",
+            SQLConf.STATE_STORE_CHECKPOINT_FORMAT_VERSION.key ->
+              failureConf.checkpointFormatVersion,
+            STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key -> 
fileManagerClassName)
+
+          testStream(aggregated, Update)(
+            StartStream(
+              checkpointLocation = checkpointDir.getAbsolutePath,
+              additionalConfs = additionalConfs),
+            AddData(inputData, 3),
+            CheckLastBatch((3, 1)),
+            AddData(inputData, 3, 2),
+            // We should categorize this error.
+            // TODO after the error is categorized, we should check error class
+            ExpectFailure[IOException] { _ => () }
+          )
+
+          injectionState.createAtomicDelayCloseRegex = Seq.empty
+
+          inputData.addData(3, 1)
+
+          // The query will restart successfully and start at the checkpoint 
after Batch 1
+          testStream(aggregated, Update)(
+            StartStream(
+              checkpointLocation = checkpointDir.getAbsolutePath,
+              additionalConfs = additionalConfs),
+            AddData(inputData, 4),
+            if (failureConf.logType == "commits") {
+              // If the failure is in the commit log, data is already 
committed. The batch will

Review Comment:
   Sorry I think I by mistake used CheckLastBatch. Using CheckNewAnswer will be 
much less confusing and I'm changing to use that.



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