Skip to content

[SPARK-51714][SS] Add Failure Ingestion test to test state store checkpoint format V2 #50508

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 12 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,23 @@ class RocksDB(
private val nativeStats = rocksDbOptions.statistics()

private val workingDir = createTempDir("workingDir")
private val fileManager = new RocksDBFileManager(dfsRootDir, createTempDir("fileManager"),

protected def CreateFileManager(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: createFileManager

dfsRootDir: String,
localTempDir: File,
hadoopConf: Configuration,
codecName: String,
loggingId: String): RocksDBFileManager = {
new RocksDBFileManager(
dfsRootDir,
localTempDir,
hadoopConf,
codecName,
loggingId = loggingId
)
}

private val fileManager = CreateFileManager(dfsRootDir, createTempDir("fileManager"),
hadoopConf, conf.compressionCodec, loggingId = loggingId)
private val byteArrayPair = new ByteArrayPair()
private val commitLatencyMs = new mutable.HashMap[String, Long]()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -31,7 +31,7 @@ import com.fasterxml.jackson.databind.annotation.JsonDeserialize
import com.fasterxml.jackson.module.scala.{ClassTagExtensions, DefaultScalaModule}
import org.apache.commons.io.{FilenameUtils, IOUtils}
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path, PathFilter}
import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter}
import org.json4s.{Formats, NoTypeHints}
import org.json4s.jackson.Serialization

Expand Down Expand Up @@ -132,8 +132,12 @@ class RocksDBFileManager(

import RocksDBImmutableFile._

protected def GetFileSystem(myDfsRootDir: String, myHadoopConf: Configuration) : FileSystem = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: getFileSystem

new Path(myDfsRootDir).getFileSystem(myHadoopConf)
}

private lazy val fm = CheckpointFileManager.create(new Path(dfsRootDir), hadoopConf)
private val fs = new Path(dfsRootDir).getFileSystem(hadoopConf)
private val fs = GetFileSystem(dfsRootDir, hadoopConf)
private val onlyZipFiles = new PathFilter {
override def accept(path: Path): Boolean = path.toString.endsWith(".zip")
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -524,13 +524,33 @@ private[sql] class RocksDBStateStoreProvider
@volatile private var stateStoreEncoding: String = _
@volatile private var stateSchemaProvider: Option[StateSchemaProvider] = _

protected def CreateRocksDB(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Could we move all these functions to start with lower case ?

dfsRootDir: String,
conf: RocksDBConf,
localRootDir: File,
hadoopConf: Configuration,
loggingId: String,
useColumnFamilies: Boolean,
enableStateStoreCheckpointIds: Boolean,
partitionId: Int = 0): RocksDB = {
new RocksDB(
dfsRootDir,
conf,
localRootDir,
hadoopConf,
loggingId,
useColumnFamilies,
enableStateStoreCheckpointIds,
partitionId)
}

private[sql] lazy val rocksDB = {
val dfsRootDir = stateStoreId.storeCheckpointLocation().toString
val storeIdStr = s"StateStoreId(opId=${stateStoreId.operatorId}," +
s"partId=${stateStoreId.partitionId},name=${stateStoreId.storeName})"
val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf)
val localRootDir = Utils.createTempDir(Utils.getLocalDir(sparkConf), storeIdStr)
new RocksDB(dfsRootDir, RocksDBConf(storeConf), localRootDir, hadoopConf, storeIdStr,
CreateRocksDB(dfsRootDir, RocksDBConf(storeConf), localRootDir, hadoopConf, storeIdStr,
useColumnFamilies, storeConf.enableStateStoreCheckpointIds, stateStoreId.partitionId)
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -988,6 +988,12 @@ object StateStore extends Logging {

/** Stop maintenance thread and reset the maintenance task */
def stopMaintenanceTask(): Unit = loadedProviders.synchronized {
stopMaintenanceTaskNoLock()
}

/** Only used for unit tests. The function doesn't hold lockloadedProviders. Calling
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: doesn't hold the loadedProviders lock

* it can work-around a deadlock condition where a maintenance task is waiting for the lock */
private[streaming] def stopMaintenanceTaskNoLock(): Unit = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: stopMaintenanceTaskWithoutLock

if (maintenanceThreadPool != null) {
maintenanceThreadPoolLock.synchronized {
maintenancePartitions.clear()
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,240 @@
/*
* 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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: lets follow the comment style ?

* stream to FailureInjectionFileSystem.delayedStreams */
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 =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

we are adding to a singleton here ?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, it is a singleton.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Updating singleton fields within this class feels a bit odd. Do you think we can refactor a bit ?

Copy link
Contributor Author

@siying siying Apr 3, 2025

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@anishshri-db it's very hard to access any of these objects from the test code, as they are created deep in the state store code as we are operating on the DB level. Static variables are the best way of communicating between unit test and file system wrapper in deep level that I can think of. What's the main concern of using singleton here?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Hmm ok - wonder if there is some way to be keep this test/class local. but if not, its fine. also cc - @HeartSaVioR - in case he has seen other patterns for similar cases before

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

If the singleton is not reused among suites, it's probably fine.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

But I'm concerned the status of object instance are "shared" and there is no reset to make sure the test is isolated.

Shall we have a clear() method to reset the status, and ensure tests relying on this object to call that method in prior?

Also, I'd feel more comfort to move the content of this file to be in RocksDBCheckpointFailureInjectionSuite, since the object instance is "unsafe" to be used among test suites. (There are some environments where multiple test suites can be executed in parallel.)

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 functions 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.
*/
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
}

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: extra newline ?

}

override def renameTempFile(srcPath: Path, dstPath: Path,
overwriteIfPossible: Boolean): Unit = {
if (FailureInjectionFileSystem.allowOverwriteInRename || !fs.exists(dstPath)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: indent seems off ?

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 */
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: could we add some more details here ?

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
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: is it close() rather than createAtomic()?

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@siying Could you please revisit this?

// `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. */
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,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: indent seems off ?

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)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

could move to same line to be consistent ?

}

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 rapper RocksDB State Store Provider that replaces FileSystem used in RocksDBFileManager
* to FailureInjectionFileSystem. */
class FailureInjectionRocksDBStateStoreProvider extends RocksDBStateStoreProvider {
override def CreateRocksDB(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: 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 made RocksDB create a RocksDBFileManager but a different FileSystem here. */
def createRocksDBWithFaultInjection(
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: any reason this has to be placed in different object / method? If this is only used once in createRocksDB, I'm not sure this is quite different from inlining.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is actually used twice. One in FailureInjectionRocksDBStateStoreProvider.createRocksDB() and one in withDBT

dfsRootDir: String,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: indent seems off ?

conf: RocksDBConf,
localRootDir: File,
hadoopConf: Configuration,
loggingId: String,
useColumnFamilies: Boolean,
enableStateStoreCheckpointIds: Boolean,
partitionId: Int): RocksDB = {
new RocksDB(
dfsRootDir,
conf = conf,
localRootDir = localRootDir,
hadoopConf = hadoopConf,
loggingId = loggingId,
useColumnFamilies = useColumnFamilies,
enableStateStoreCheckpointIds = enableStateStoreCheckpointIds,
partitionId = partitionId
) {
override def CreateFileManager(
dfsRootDir: String,
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: indent

localTempDir: File,
hadoopConf: Configuration,
codecName: String,
loggingId: String): RocksDBFileManager = {
new RocksDBFileManager(
dfsRootDir,
localTempDir,
hadoopConf,
codecName,
loggingId = loggingId
) {
override def GetFileSystem(
myDfsRootDir: String,
myHadoopConf: Configuration): FileSystem = {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

nit: indent

new FailureInjectionFileSystem(new Path(myDfsRootDir).getFileSystem(myHadoopConf))
}
}
}
}
}
}
Loading