siying commented on code in PR #50508: URL: https://github.com/apache/spark/pull/50508#discussion_r2030026177
########## 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 come + * 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() Review Comment: We don't have it. Maybe do it as a follow up when we have time later. -- 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