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


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala:
##########
@@ -0,0 +1,560 @@
+/*
+ * 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.state
+
+import java.io._
+
+import scala.language.implicitConversions
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.{SparkConf, SparkException}
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.count
+import org.apache.spark.sql.internal.SQLConf
+import 
org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS
+import org.apache.spark.sql.streaming._
+import org.apache.spark.sql.streaming.OutputMode.Update
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.tags.SlowSQLTest
+import org.apache.spark.util.Utils
+
+
+@SlowSQLTest
+/**
+ * Test suite to inject some failures in RocksDB checkpoint */
+class RocksDBCheckpointFailureInjectionSuite extends StreamTest
+  with SharedSparkSession {
+  override protected def sparkConf: SparkConf = {
+    super.sparkConf
+      .set(SQLConf.STATE_STORE_PROVIDER_CLASS, 
classOf[RocksDBStateStoreProvider].getName)
+
+  }
+
+  override def beforeEach(): Unit = {
+    super.beforeEach()
+    FailureInjectionFileSystem.failPreCopyFromLocalFileNameRegex = Seq.empty
+    FailureInjectionFileSystem.failureCreateAtomicRegex = Seq.empty
+    FailureInjectionFileSystem.shouldFailExist = false
+  }
+
+  implicit def toArray(str: String): Array[Byte] = if (str != null) 
str.getBytes else null
+
+  Seq(false, true).foreach { ifEnableStateStoreCheckpointIds =>
+    test(
+      "Basic RocksDB SST File Upload Failure Handling" +
+      s"ifEnableStateStoreCheckpointIds = $ifEnableStateStoreCheckpointIds") {
+      val fmClass = "org.apache.spark.sql.execution.streaming.state." +
+        "FailureInjectionCheckpointFileManager"
+      val hadoopConf = new Configuration()
+      hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, 
fmClass)
+      withTempDir { remoteDir =>
+        withSQLConf(
+          RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled" -> "false") {
+          val conf = RocksDBConf(StateStoreConf(SQLConf.get))
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 0,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = ifEnableStateStoreCheckpointIds) { 
db =>
+            db.put("version", "1.1")
+            val checkpointId1 = commitAndGetCheckpointId(db)
+
+            FailureInjectionFileSystem.failPreCopyFromLocalFileNameRegex = 
Seq(".*sst")
+            db.put("version", "2.1")
+            var checkpointId2: Option[String] = None
+            intercept[IOException] {
+              checkpointId2 = commitAndGetCheckpointId(db)
+            }
+
+            db.load(1, checkpointId1)
+
+            FailureInjectionFileSystem.failPreCopyFromLocalFileNameRegex = 
Seq.empty
+            // When ifEnableStateStoreCheckpointIds is true, checkpointId is 
not available
+            // to load version 2. If we use None, it will throw a Runtime 
error. We probably
+            // should categorize this error.
+            if (!ifEnableStateStoreCheckpointIds) {
+              val ex = intercept[SparkException] {
+                db.load(2, checkpointId2)
+              }
+              checkError(
+                ex,
+                condition = 
"CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE",
+                parameters = Map(
+                  "fileToRead" -> s"$remoteDir/2.changelog"

Review Comment:
   Yes, but I think that's always the error we throw, even if changelog is 
disabled.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/FailureInjectionCheckpointFileManager.scala:
##########
@@ -0,0 +1,254 @@
+/*
+ * 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.state
+
+import java.io._
+import java.net.URI
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs._
+import org.apache.hadoop.fs.permission.FsPermission
+import org.apache.hadoop.util.Progressable
+
+import org.apache.spark.internal.Logging
+import 
org.apache.spark.sql.execution.streaming.CheckpointFileManager.{CancellableFSDataOutputStream,
 RenameBasedFSDataOutputStream}
+import 
org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager
+
+/**
+ * A wrapper file output stream that will throw exception in close() and put 
the underlying
+ * stream to FailureInjectionFileSystem.delayedStreams
+ * @param stream stream to be wrapped
+ */
+class DelayCloseFSDataOutputStreamWrapper(stream: 
CancellableFSDataOutputStream)
+  extends CancellableFSDataOutputStream(stream.getWrappedStream) with Logging {
+  val originalStream: CancellableFSDataOutputStream = stream
+
+  var closed: Boolean = false
+
+  override def close(): Unit = {
+    if (!closed) {
+      closed = true
+      FailureInjectionFileSystem.delayedStreams =
+        FailureInjectionFileSystem.delayedStreams :+ originalStream
+      throw new IOException("Fake File Stream Close Failure")
+    }
+  }
+
+  /** Cancel is not needed in unit tests */
+  override def cancel(): Unit = {}
+}
+
+/**
+ * A wrapper checkpoint file manager that might inject failures in some 
function calls.
+ * Used in unit tests to simulate failure scenarios.
+ * This can be put into SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS to 
provide failure
+ * injection behavior.
+ *
+ * @param path The path to the checkpoint directory, passing to the parent 
class
+ * @param hadoopConf  hadoop conf that will be passed to the parent class
+ */
+class FailureInjectionCheckpointFileManager(path: Path, hadoopConf: 
Configuration)
+  extends FileSystemBasedCheckpointFileManager(path, hadoopConf) with Logging {
+
+  override def createAtomic(path: Path,
+                            overwriteIfPossible: Boolean): 
CancellableFSDataOutputStream = {
+    FailureInjectionFileSystem.failureCreateAtomicRegex.foreach { pattern =>
+      if (path.toString.matches(pattern)) {
+        throw new IOException("Fake File System Create Atomic Failure")
+      }
+    }
+
+    var shouldDelay = false
+    FailureInjectionFileSystem.createAtomicDelayCloseRegex.foreach { pattern =>
+      if (path.toString.matches(pattern)) {
+        shouldDelay = true
+      }
+    }
+    val ret = new RenameBasedFSDataOutputStream(this, path, 
overwriteIfPossible)
+    if (shouldDelay) {
+      new DelayCloseFSDataOutputStreamWrapper(ret)
+    } else {
+      ret
+    }
+  }
+
+  override def renameTempFile(srcPath: Path, dstPath: Path, 
overwriteIfPossible: Boolean): Unit = {
+    if (FailureInjectionFileSystem.allowOverwriteInRename || 
!fs.exists(dstPath)) {
+      super.renameTempFile(srcPath, dstPath, overwriteIfPossible)
+    } else {
+      logWarning(s"Skip renaming temp file $srcPath to $dstPath because it 
already exists.")
+    }
+  }
+
+  override def list(path: Path, filter: PathFilter): Array[FileStatus] = {
+    super.list(path, filter)
+  }
+
+  override def exists(path: Path): Boolean = {
+    if (FailureInjectionFileSystem.shouldFailExist) {
+      throw new IOException("Fake File Exists Failure")
+    }
+    super.exists(path)
+  }
+}
+
+/**
+ * Contains a list of variables for failure ingestion conditions.
+ * These are singleton instances accessed by all instances of 
FailureInjectionCheckpointFileManager
+ * and FailureInjectionFileSystem. This allows a unit test to have a global 
control of failure
+ * and access to the delayed streams.
+ */
+object FailureInjectionFileSystem {
+  // File names matching this regex will cause the copyFromLocalFile to fail
+  var failPreCopyFromLocalFileNameRegex: Seq[String] = Seq.empty
+  // File names matching this regex will cause the createAtomic to fail and 
put the streams in
+  // `delayedStreams`
+  var createAtomicDelayCloseRegex: Seq[String] = Seq.empty
+  // File names matching this regex will cause the createAtomic() to fail
+  var failureCreateAtomicRegex: Seq[String] = Seq.empty
+  // If true, Exists() call will fail
+  var shouldFailExist: Boolean = false
+  // If true, simulate a case where rename() will not overwrite an existing 
file.
+  var allowOverwriteInRename: Boolean = true
+
+  // List of streams that are delayed in close() based on 
`createAtomicDelayCloseRegex`
+  var delayedStreams: Seq[CancellableFSDataOutputStream] = Seq.empty
+}
+
+/**
+ * A wrapper FileSystem that inject some failures. This class can used to 
replace the
+ * FileSystem in RocksDBFileManager.
+ * @param innerFs  the FileSystem to be wrapped
+ */
+class FailureInjectionFileSystem(innerFs: FileSystem) extends FileSystem {
+
+  override def getConf: Configuration = innerFs.getConf
+
+  override def mkdirs(f: Path, permission: FsPermission): Boolean = 
innerFs.mkdirs(f, permission)
+
+  override def rename(src: Path, dst: Path): Boolean = innerFs.rename(src, dst)
+
+  override def getUri: URI = innerFs.getUri
+
+  override def open(f: Path, bufferSize: Int): FSDataInputStream = 
innerFs.open(f, bufferSize)
+
+  override def create(
+      f: Path,
+      permission: FsPermission,
+      overwrite: Boolean,
+      bufferSize: Int,
+      replication: Short,
+      blockSize: Long,
+      progress: Progressable): FSDataOutputStream =
+    innerFs.create(f, permission, overwrite, bufferSize, replication, 
blockSize, progress)
+
+  override def append(f: Path, bufferSize: Int, progress: Progressable): 
FSDataOutputStream =
+    innerFs.append(f, bufferSize, progress)
+
+  override def delete(f: Path, recursive: Boolean): Boolean = 
innerFs.delete(f, recursive)
+
+  override def listStatus(f: Path): Array[FileStatus] = innerFs.listStatus(f)
+
+  override def setWorkingDirectory(new_dir: Path): Unit = 
innerFs.setWorkingDirectory(new_dir)
+
+  override def getWorkingDirectory: Path = innerFs.getWorkingDirectory
+
+  override def getFileStatus(f: Path): FileStatus = innerFs.getFileStatus(f)
+
+  override def copyFromLocalFile(src: Path, dst: Path): Unit = {
+    FailureInjectionFileSystem.failPreCopyFromLocalFileNameRegex.foreach { 
pattern =>
+      if (src.toString.matches(pattern)) {
+        throw new IOException(s"Injected failure due to source path matching 
pattern: $pattern")
+      }
+    }
+
+    innerFs.copyFromLocalFile(src, dst)
+  }
+}
+
+/**
+ * A wrapper RocksDB State Store Provider that replaces FileSystem used in 
RocksDBFileManager
+ * to FailureInjectionFileSystem.
+ */
+class FailureInjectionRocksDBStateStoreProvider extends 
RocksDBStateStoreProvider {
+  override def createRocksDB(
+      dfsRootDir: String,
+      conf: RocksDBConf,
+      localRootDir: File,
+      hadoopConf: Configuration,
+      loggingId: String,
+      useColumnFamilies: Boolean,
+      enableStateStoreCheckpointIds: Boolean,
+      partitionId: Int): RocksDB = {
+    FailureInjectionRocksDBStateStoreProvider.createRocksDBWithFaultInjection(
+      dfsRootDir,
+      conf,
+      localRootDir,
+      hadoopConf,
+      loggingId,
+      useColumnFamilies,
+      enableStateStoreCheckpointIds,
+      partitionId)
+  }
+}
+
+object FailureInjectionRocksDBStateStoreProvider {
+  /**
+   * RocksDBFieManager is created by RocksDB class where it creates a default 
FileSystem.
+   * We make RocksDB create a RocksDBFileManager that uses a different 
FileSystem here.
+   * */
+  def createRocksDBWithFaultInjection(

Review Comment:
   This is actually used twice. One in 
FailureInjectionRocksDBStateStoreProvider.createRocksDB() and one in withDB[T]()



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBCheckpointFailureInjectionSuite.scala:
##########
@@ -0,0 +1,560 @@
+/*
+ * 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.state
+
+import java.io._
+
+import scala.language.implicitConversions
+
+import org.apache.hadoop.conf.Configuration
+
+import org.apache.spark.{SparkConf, SparkException}
+import org.apache.spark.sql.execution.streaming.MemoryStream
+import org.apache.spark.sql.functions.count
+import org.apache.spark.sql.internal.SQLConf
+import 
org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS
+import org.apache.spark.sql.streaming._
+import org.apache.spark.sql.streaming.OutputMode.Update
+import org.apache.spark.sql.test.SharedSparkSession
+import org.apache.spark.tags.SlowSQLTest
+import org.apache.spark.util.Utils
+
+
+@SlowSQLTest
+/**
+ * Test suite to inject some failures in RocksDB checkpoint */
+class RocksDBCheckpointFailureInjectionSuite extends StreamTest
+  with SharedSparkSession {
+  override protected def sparkConf: SparkConf = {
+    super.sparkConf
+      .set(SQLConf.STATE_STORE_PROVIDER_CLASS, 
classOf[RocksDBStateStoreProvider].getName)
+
+  }
+
+  override def beforeEach(): Unit = {
+    super.beforeEach()
+    FailureInjectionFileSystem.failPreCopyFromLocalFileNameRegex = Seq.empty
+    FailureInjectionFileSystem.failureCreateAtomicRegex = Seq.empty
+    FailureInjectionFileSystem.shouldFailExist = false
+  }
+
+  implicit def toArray(str: String): Array[Byte] = if (str != null) 
str.getBytes else null
+
+  Seq(false, true).foreach { ifEnableStateStoreCheckpointIds =>
+    test(
+      "Basic RocksDB SST File Upload Failure Handling" +
+      s"ifEnableStateStoreCheckpointIds = $ifEnableStateStoreCheckpointIds") {
+      val fmClass = "org.apache.spark.sql.execution.streaming.state." +
+        "FailureInjectionCheckpointFileManager"
+      val hadoopConf = new Configuration()
+      hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, 
fmClass)
+      withTempDir { remoteDir =>
+        withSQLConf(
+          RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled" -> "false") {
+          val conf = RocksDBConf(StateStoreConf(SQLConf.get))
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 0,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = ifEnableStateStoreCheckpointIds) { 
db =>
+            db.put("version", "1.1")
+            val checkpointId1 = commitAndGetCheckpointId(db)
+
+            FailureInjectionFileSystem.failPreCopyFromLocalFileNameRegex = 
Seq(".*sst")
+            db.put("version", "2.1")
+            var checkpointId2: Option[String] = None
+            intercept[IOException] {
+              checkpointId2 = commitAndGetCheckpointId(db)
+            }
+
+            db.load(1, checkpointId1)
+
+            FailureInjectionFileSystem.failPreCopyFromLocalFileNameRegex = 
Seq.empty
+            // When ifEnableStateStoreCheckpointIds is true, checkpointId is 
not available
+            // to load version 2. If we use None, it will throw a Runtime 
error. We probably
+            // should categorize this error.
+            if (!ifEnableStateStoreCheckpointIds) {
+              val ex = intercept[SparkException] {
+                db.load(2, checkpointId2)
+              }
+              checkError(
+                ex,
+                condition = 
"CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE",
+                parameters = Map(
+                  "fileToRead" -> s"$remoteDir/2.changelog"
+                )
+              )
+            }
+
+            db.load(0)
+            FailureInjectionFileSystem.shouldFailExist = true
+            intercept[IOException] {
+              db.load(1, checkpointId1)
+            }
+          }
+        }
+      }
+    }
+  }
+
+  Seq(false, true).foreach { ifEnableStateStoreCheckpointIds =>
+    test(
+      "Basic RocksDB Zip File Upload Failure Handling " +
+        s"ifEnableStateStoreCheckpointIds = $ifEnableStateStoreCheckpointIds") 
{
+      val fmClass = "org.apache.spark.sql.execution.streaming.state." +
+        "FailureInjectionCheckpointFileManager"
+      val hadoopConf = new Configuration()
+      hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, 
fmClass)
+      withTempDir { remoteDir =>
+        withSQLConf(
+          RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled" -> "false") {
+          val conf = RocksDBConf(StateStoreConf(SQLConf.get))
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 0,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = ifEnableStateStoreCheckpointIds) { 
db =>
+            db.put("version", "1.1")
+            val checkpointId1 = commitAndGetCheckpointId(db)
+
+            db.load(1, checkpointId1)
+            FailureInjectionFileSystem.failureCreateAtomicRegex = Seq(".*zip")
+            db.put("version", "2.1")
+            var checkpointId2: Option[String] = None
+            intercept[IOException] {
+              checkpointId2 = commitAndGetCheckpointId(db)
+            }
+
+            db.load(1, checkpointId1)
+
+            FailureInjectionFileSystem.failureCreateAtomicRegex = Seq.empty
+
+            // When ifEnableStateStoreCheckpointIds is true, checkpointId is 
not available
+            // to load version 2. If we use None, it will throw a Runtime 
error. We probably
+            // should categorize this error.
+            if (!ifEnableStateStoreCheckpointIds) {
+              val ex = intercept[SparkException] {
+                db.load(2, checkpointId2)
+              }
+              checkError(
+                ex,
+                condition = 
"CANNOT_LOAD_STATE_STORE.CANNOT_READ_STREAMING_STATE_FILE",
+                parameters = Map(
+                  "fileToRead" -> s"$remoteDir/2.changelog"
+                )
+              )
+            }
+
+            db.load(0)
+            FailureInjectionFileSystem.shouldFailExist = true
+            intercept[IOException] {
+              db.load(1, checkpointId1)
+            }
+          }
+        }
+      }
+    }
+  }
+
+
+  /**
+   * This test is to simulate the case where a previous task had connectivity 
problem that couldn't
+   * be killed or write zip file. Only after the later one is successfully 
committed, it comes back
+   * and write the zip file.
+   */
+  Seq(true, false).foreach { ifEnableStateStoreCheckpointIds =>
+    test(
+      "Zip File Overwritten by Previous Task Checkpoint " +
+      s"ifEnableStateStoreCheckpointIds = $ifEnableStateStoreCheckpointIds") {
+      val fmClass = "org.apache.spark.sql.execution.streaming.state." +
+        "FailureInjectionCheckpointFileManager"
+      val hadoopConf = new Configuration()
+      hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, 
fmClass)
+      withTempDir { remoteDir =>
+        withSQLConf(
+          RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled" -> "false") {
+          val conf = RocksDBConf(StateStoreConf(SQLConf.get))
+
+          var checkpointId2: Option[String] = None
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 0,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = ifEnableStateStoreCheckpointIds) { 
db =>
+            db.put("version", "1.1")
+            val checkpointId1 = commitAndGetCheckpointId(db)
+
+            db.load(1, checkpointId1)
+            FailureInjectionFileSystem.createAtomicDelayCloseRegex = 
Seq(".*zip")
+            db.put("version", "2.1")
+
+            intercept[IOException] {
+              commitAndGetCheckpointId(db)
+            }
+
+            FailureInjectionFileSystem.createAtomicDelayCloseRegex = Seq.empty
+
+            db.load(1, checkpointId1)
+
+            db.put("version", "2.2")
+            checkpointId2 = commitAndGetCheckpointId(db)
+
+            assert(FailureInjectionFileSystem.delayedStreams.nonEmpty)
+            FailureInjectionFileSystem.delayedStreams.foreach(_.close())
+          }
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 2,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = ifEnableStateStoreCheckpointIds,
+            checkpointId = checkpointId2) { db =>
+            if (ifEnableStateStoreCheckpointIds) {
+              assert(new String(db.get("version"), "UTF-8") == "2.2")
+            } else {
+              // Assuming previous 2.zip overwrites, we should see the 
previous value.
+              // This validation isn't necessary here but we just would like 
to make sure
+              // FailureInjectionCheckpointFileManager has correct behavior -- 
 allows zip files
+              // to be delayed to be written, so that the test for
+              // ifEnableStateStoreCheckpointIds = trrue is valid.
+              assert(new String(db.get("version"), "UTF-8") == "2.1")
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * This test is to simulate the case where a previous task had connectivity 
problem that couldn't
+   * be killed or write changelog file. Only after the later one is 
successfully committed, it comes
+   * back and write the changelog file.
+   *  */
+  Seq(false, true).foreach { ifEnableStateStoreCheckpointIds =>
+    test(
+      "Changelog File Overwritten by Previous Task With Changelog Checkpoint " 
+
+      s"ifEnableStateStoreCheckpointIds = $ifEnableStateStoreCheckpointIds") {
+      val fmClass = "org.apache.spark.sql.execution.streaming.state." +
+        "FailureInjectionCheckpointFileManager"
+      val hadoopConf = new Configuration()
+      hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, 
fmClass)
+      withTempDir { remoteDir =>
+        withSQLConf(
+          RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled" -> "true",
+          SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key -> "5") {
+          val conf = RocksDBConf(StateStoreConf(SQLConf.get))
+
+          var checkpointId2: Option[String] = None
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 0,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = ifEnableStateStoreCheckpointIds) { 
db =>
+            db.put("version", "1.1")
+            val checkpointId1 = commitAndGetCheckpointId(db)
+
+            FailureInjectionFileSystem.createAtomicDelayCloseRegex = 
Seq(".*/2.*changelog")
+
+            db.load(1, checkpointId1)
+            db.put("version", "2.1")
+            intercept[IOException] {
+              commitAndGetCheckpointId(db)
+            }
+
+            FailureInjectionFileSystem.createAtomicDelayCloseRegex = Seq.empty
+
+            db.load(1, checkpointId1)
+
+            db.put("version", "2.2")
+            checkpointId2 = commitAndGetCheckpointId(db)
+
+            assert(FailureInjectionFileSystem.delayedStreams.nonEmpty)
+            FailureInjectionFileSystem.delayedStreams.foreach(_.close())
+
+            db.load(1, checkpointId1)
+            db.load(2, checkpointId2)
+          }
+
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 2,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = ifEnableStateStoreCheckpointIds,
+            checkpointId = checkpointId2) { db =>
+            if (ifEnableStateStoreCheckpointIds) {
+              assert(new String(db.get("version"), "UTF-8") == "2.2")
+            } else {
+              // This check is not necessary. But we would like to validate 
the behavior of
+              // FailureInjectionFileSystem to ensure the 
ifEnableStateStoreCheckpointIds = true
+              // case is valid.
+              assert(new String(db.get("version"), "UTF-8") == "2.1")
+            }
+          }
+        }
+      }
+    }
+  }
+
+  /**
+   * This test is to simulate the case where
+   * 1. There is a snapshot checkpoint scheduled
+   * 2. The batch eventually failed
+   * 3. Query is retried and moved forward
+   * 4. The snapshot checkpoint succeeded
+   * In checkpoint V2, this snapshot shouldn't take effect. Otherwise, it will 
break the strong
+   * consistency guaranteed by V2.
+   */
+  test("Delay Snapshot V2") {
+    val fmClass = "org.apache.spark.sql.execution.streaming.state." +
+      "FailureInjectionCheckpointFileManager"
+    val hadoopConf = new Configuration()
+    hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, fmClass)
+    withTempDir { remoteDir =>
+      withSQLConf(
+        RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled" -> "true",
+        SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key -> "2") {
+        val conf = RocksDBConf(StateStoreConf(SQLConf.get))
+        var checkpointId3: Option[String] = None
+        withDB(
+          remoteDir.getAbsolutePath,
+          version = 0,
+          conf = conf,
+          hadoopConf = hadoopConf,
+          enableStateStoreCheckpointIds = true) { db =>
+          db.put("version", "1.1")
+          val checkpointId1 = commitAndGetCheckpointId(db)
+
+          withDB(
+            remoteDir.getAbsolutePath,
+            version = 1,
+            conf = conf,
+            hadoopConf = hadoopConf,
+            enableStateStoreCheckpointIds = true,
+            checkpointId = checkpointId1) { db2 =>
+            db2.put("version", "2.1")
+            db2.commit()
+
+            db.load(1, checkpointId1)
+            db.put("version", "2.2")
+            val checkpointId2 = commitAndGetCheckpointId(db)
+
+            db.load(2, checkpointId2)
+            db.put("foo", "bar")
+            checkpointId3 = commitAndGetCheckpointId(db)
+
+            db2.doMaintenance()
+          }
+        }
+        withDB(
+          remoteDir.getAbsolutePath,
+          version = 3,
+          conf = conf,
+          hadoopConf = hadoopConf,
+          enableStateStoreCheckpointIds = true,
+          checkpointId = checkpointId3) { db =>
+          assert(new String(db.get("version"), "UTF-8") == "2.2")
+          assert(new String(db.get("foo"), "UTF-8") == "bar")
+        }
+      }
+    }
+  }
+
+  import testImplicits._
+
+  /**
+   * An integrated test where a previous changelog from a failed batch come 
back and finish
+   * writing. In checkpoint V2, this changelog should be ignored.
+   * Test it with both file renaming overwrite and not renaming overwrite.
+   */
+  Seq(false, true).foreach { ifAllowRenameOverwrite =>
+    test(s"Job failure with changelog shows up ifAllowRenameOverwrite = 
$ifAllowRenameOverwrite") {
+      val fmClass = "org.apache.spark.sql.execution.streaming.state." +
+        "FailureInjectionCheckpointFileManager"
+      val hadoopConf = new Configuration()
+      hadoopConf.set(STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, 
fmClass)
+      val rocksdbChangelogCheckpointingConfKey =
+        RocksDBConf.ROCKSDB_SQL_CONF_NAME_PREFIX + 
".changelogCheckpointing.enabled"
+      FailureInjectionFileSystem.allowOverwriteInRename = 
ifAllowRenameOverwrite
+      withTempDir { checkpointDir =>
+        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)]
+
+          FailureInjectionFileSystem.createAtomicDelayCloseRegex = 
Seq(".*/2_.*changelog")
+
+          testStream(aggregated, Update)(
+            StartStream(checkpointLocation = checkpointDir.getAbsolutePath,
+              additionalConfs = Map(
+                rocksdbChangelogCheckpointingConfKey -> "true",
+                SQLConf.STATE_STORE_CHECKPOINT_FORMAT_VERSION.key -> "2",
+                STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key -> 
fmClass)),

Review Comment:
   I copied it from other test suites. tests such as 
`CheckpointFileManagerSuite`, `RocksDBSuite` and other several tests are all 
doing this.



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