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


##########
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:
   OK that is odd. @anishshri-db Would you mind taking a look?



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