Skip to content

Commit

Permalink
Merge pull request #1 from anishshri-db/story/state-v2-changelog
Browse files Browse the repository at this point in the history
Add support for changelog checkpointing with new operator
  • Loading branch information
anishshri-db authored Dec 14, 2023
2 parents 3a93b92 + 1ff70db commit 16f7c4c
Show file tree
Hide file tree
Showing 7 changed files with 413 additions and 163 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -144,12 +144,6 @@ class RocksDB(
@volatile private var numKeysOnWritingVersion = 0L
@volatile private var fileManagerMetrics = RocksDBFileManagerMetrics.EMPTY_METRICS

// TODO: support changelog checkpointing with column families
if (useColumnFamilies && enableChangelogCheckpointing) {
throw new RuntimeException("Changelog checkpointing is not supported with multiple " +
"column families")
}

// SPARK-46249 - Keep track of recorded metrics per version which can be used for querying later
// Updates and access to recordedMetrics are protected by the DB instance lock
@GuardedBy("acquireLock")
Expand Down Expand Up @@ -205,7 +199,7 @@ class RocksDB(
if (enableChangelogCheckpointing && !readOnly) {
// Make sure we don't leak resource.
changelogWriter.foreach(_.abort())
changelogWriter = Some(fileManager.getChangeLogWriter(version + 1))
changelogWriter = Some(fileManager.getChangeLogWriter(version + 1, useColumnFamilies))
}
this
}
Expand All @@ -217,12 +211,18 @@ class RocksDB(
for (v <- loadedVersion + 1 to endVersion) {
var changelogReader: StateStoreChangelogReader = null
try {
changelogReader = fileManager.getChangelogReader(v)
changelogReader.foreach { case (key, value) =>
if (value != null) {
put(key, value)
} else {
remove(key)
changelogReader = fileManager.getChangelogReader(v, useColumnFamilies)
changelogReader.foreach { case (recordType, key, value, colFamilyName) =>
if (useColumnFamilies && !checkColFamilyExists(colFamilyName)) {
createColFamilyIfAbsent(colFamilyName)
}

recordType match {
case RecordType.PUT_RECORD =>
put(key, value, colFamilyName)

case RecordType.DELETE_RECORD =>
remove(key, colFamilyName)
}
}
} finally {
Expand Down Expand Up @@ -289,6 +289,7 @@ class RocksDB(
}
}
db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value)
changelogWriter.foreach(_.put(key, value, colFamilyName))
} else {
if (conf.trackTotalNumberOfRows) {
val oldValue = db.get(readOptions, key)
Expand Down Expand Up @@ -319,6 +320,7 @@ class RocksDB(
}
}
db.delete(colFamilyNameToHandleMap(colFamilyName), writeOptions, key)
changelogWriter.foreach(_.delete(key, colFamilyName))
} else {
if (conf.trackTotalNumberOfRows) {
val value = db.get(readOptions, key)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -149,21 +149,31 @@ class RocksDBFileManager(

@volatile private var rootDirChecked: Boolean = false

def getChangeLogWriter(version: Long): StateStoreChangelogWriter = {
def getChangeLogWriter(version: Long,
useColumnFamilies: Boolean = false): StateStoreChangelogWriter = {
val changelogFile = dfsChangelogFile(version)
if (!rootDirChecked) {
val rootDir = new Path(dfsRootDir)
if (!fm.exists(rootDir)) fm.mkdirs(rootDir)
rootDirChecked = true
}
val changelogWriter = new StateStoreChangelogWriter(fm, changelogFile, codec)
val changelogWriter = if (useColumnFamilies) {
new StateStoreChangelogWriterV2(fm, changelogFile, codec)
} else {
new StateStoreChangelogWriterV1(fm, changelogFile, codec)
}
changelogWriter
}

// Get the changelog file at version
def getChangelogReader(version: Long): StateStoreChangelogReader = {
def getChangelogReader(version: Long,
useColumnFamilies: Boolean = false): StateStoreChangelogReader = {
val changelogFile = dfsChangelogFile(version)
new StateStoreChangelogReader(fm, changelogFile, codec)
if (useColumnFamilies) {
new StateStoreChangelogReaderV2(fm, changelogFile, codec)
} else {
new StateStoreChangelogReaderV1(fm, changelogFile, codec)
}
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -33,12 +33,20 @@ import org.apache.spark.sql.execution.streaming.CheckpointFileManager.Cancellabl
import org.apache.spark.util.NextIterator

/**
* Write changes to the key value state store instance to a changelog file.
* There are 2 types of records, put and delete.
* A put record is written as: | key length | key content | value length | value content |
* A delete record is written as: | key length | key content | -1 |
* Write an Int -1 to signal the end of file.
* The overall changelog format is: | put record | delete record | ... | put record | -1 |
* Enum used to write record types to changelog files used with RocksDBStateStoreProvider.
*/
object RecordType extends Enumeration {
type RecordType = Value

val PUT_RECORD = Value("put_record")
val DELETE_RECORD = Value("delete_record")
}

/**
* Base class for state store changelog writer
* @param fm - checkpoint file manager used to manage streaming query checkpoint
* @param file - name of file to use to write changelog
* @param compressionCodec - compression method using for writing changelog file
*/
class StateStoreChangelogWriter(
fm: CheckpointFileManager,
Expand All @@ -52,25 +60,27 @@ class StateStoreChangelogWriter(

private var backingFileStream: CancellableFSDataOutputStream =
fm.createAtomic(file, overwriteIfPossible = true)
private var compressedStream: DataOutputStream = compressStream(backingFileStream)
protected var compressedStream: DataOutputStream = compressStream(backingFileStream)
var size = 0

def put(key: Array[Byte], value: Array[Byte]): Unit = {
assert(compressedStream != null)
compressedStream.writeInt(key.length)
compressedStream.write(key)
compressedStream.writeInt(value.length)
compressedStream.write(value)
size += 1
throw new UnsupportedOperationException("Operation not supported on base changelog writer " +
"implementation")
}

def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
throw new UnsupportedOperationException("Operation not supported on base changelog writer " +
"implementation")
}

def delete(key: Array[Byte]): Unit = {
assert(compressedStream != null)
compressedStream.writeInt(key.length)
compressedStream.write(key)
// -1 in the value field means record deletion.
compressedStream.writeInt(-1)
size += 1
throw new UnsupportedOperationException("Operation not supported on base changelog writer " +
"implementation")
}

def delete(key: Array[Byte], colFamilyName: String): Unit = {
throw new UnsupportedOperationException("Operation not supported on base changelog writer " +
"implementation")
}

def abort(): Unit = {
Expand Down Expand Up @@ -109,18 +119,94 @@ class StateStoreChangelogWriter(
}
}

/**
* Write changes to the key value state store instance to a changelog file.
* There are 2 types of records, put and delete.
* A put record is written as: | key length | key content | value length | value content |
* A delete record is written as: | key length | key content | -1 |
* Write an Int -1 to signal the end of file.
* The overall changelog format is: | put record | delete record | ... | put record | -1 |
*/
class StateStoreChangelogWriterV1(
fm: CheckpointFileManager,
file: Path,
compressionCodec: CompressionCodec)
extends StateStoreChangelogWriter(fm, file, compressionCodec) {

override def put(key: Array[Byte], value: Array[Byte]): Unit = {
assert(compressedStream != null)
compressedStream.writeInt(key.size)
compressedStream.write(key)
compressedStream.writeInt(value.size)
compressedStream.write(value)
size += 1
}

override def delete(key: Array[Byte]): Unit = {
assert(compressedStream != null)
compressedStream.writeInt(key.size)
compressedStream.write(key)
// -1 in the value field means record deletion.
compressedStream.writeInt(-1)
size += 1
}
}

/**
* Read an iterator of change record from the changelog file.
* A record is represented by ByteArrayPair(key: Array[Byte], value: Array[Byte])
* A put record is returned as a ByteArrayPair(key, value)
* A delete record is return as a ByteArrayPair(key, null)
* Write changes to the key value state store instance to a changelog file.
* There are 2 types of records, put and delete.
* A put record is written as: | record type | key length
* | key content | value length | value content | col family name length | col family name | -1 |
* A delete record is written as: | record type | key length | key content | -1
* | col family name length | col family name | -1 |
* Write an Int -1 to signal the end of file.
* The overall changelog format is: | put record | delete record | ... | put record | -1 |
*/
class StateStoreChangelogWriterV2(
fm: CheckpointFileManager,
file: Path,
compressionCodec: CompressionCodec)
extends StateStoreChangelogWriter(fm, file, compressionCodec) {

override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = {
assert(compressedStream != null)
compressedStream.writeInt(RecordType.PUT_RECORD.toString.getBytes.size)
compressedStream.write(RecordType.PUT_RECORD.toString.getBytes)
compressedStream.writeInt(key.size)
compressedStream.write(key)
compressedStream.writeInt(value.size)
compressedStream.write(value)
compressedStream.writeInt(colFamilyName.getBytes.size)
compressedStream.write(colFamilyName.getBytes)
size += 1
}

override def delete(key: Array[Byte], colFamilyName: String): Unit = {
assert(compressedStream != null)
compressedStream.writeInt(RecordType.DELETE_RECORD.toString.getBytes.size)
compressedStream.write(RecordType.DELETE_RECORD.toString.getBytes)
compressedStream.writeInt(key.size)
compressedStream.write(key)
// -1 in the value field means record deletion.
compressedStream.writeInt(-1)
compressedStream.writeInt(colFamilyName.getBytes.size)
compressedStream.write(colFamilyName.getBytes)
size += 1
}
}

/**
* Base class for state store changelog reader
* @param fm - checkpoint file manager used to manage streaming query checkpoint
* @param fileToRead - name of file to use to read changelog
* @param compressionCodec - de-compression method using for reading changelog file
*/
class StateStoreChangelogReader(
fm: CheckpointFileManager,
fileToRead: Path,
compressionCodec: CompressionCodec)
extends NextIterator[(Array[Byte], Array[Byte])] with Logging {
extends NextIterator[(RecordType.Value, Array[Byte], Array[Byte], String)]
with Logging {

private def decompressStream(inputStream: DataInputStream): DataInputStream = {
val compressed = compressionCodec.compressedInputStream(inputStream)
Expand All @@ -133,11 +219,30 @@ class StateStoreChangelogReader(
case f: FileNotFoundException =>
throw QueryExecutionErrors.failedToReadStreamingStateFileError(fileToRead, f)
}
private val input: DataInputStream = decompressStream(sourceStream)
protected val input: DataInputStream = decompressStream(sourceStream)

def close(): Unit = { if (input != null) input.close() }

override def getNext(): (Array[Byte], Array[Byte]) = {
override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
throw new UnsupportedOperationException("Iterator operations not supported on base " +
"changelog reader implementation")
}
}

/**
* Read an iterator of change record from the changelog file.
* A record is represented by ByteArrayPair(recordType: RecordType.Value,
* key: Array[Byte], value: Array[Byte], colFamilyName: String)
* A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
* A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
*/
class StateStoreChangelogReaderV1(
fm: CheckpointFileManager,
fileToRead: Path,
compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead,
compressionCodec) {

override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
val keySize = input.readInt()
// A -1 key size mean end of file.
if (keySize == -1) {
Expand All @@ -153,12 +258,70 @@ class StateStoreChangelogReader(
val valueSize = input.readInt()
if (valueSize < 0) {
// A deletion record
(keyBuffer, null)
(RecordType.DELETE_RECORD, keyBuffer, null, StateStore.DEFAULT_COL_FAMILY_NAME)
} else {
val valueBuffer = new Array[Byte](valueSize)
ByteStreams.readFully(input, valueBuffer, 0, valueSize)
// A put record.
(keyBuffer, valueBuffer)
(RecordType.PUT_RECORD, keyBuffer, valueBuffer, StateStore.DEFAULT_COL_FAMILY_NAME)
}
}
}
}

/**
* Read an iterator of change record from the changelog file.
* A record is represented by ByteArrayPair(recordType: RecordType.Value,
* key: Array[Byte], value: Array[Byte], colFamilyName: String)
* A put record is returned as a ByteArrayPair(recordType, key, value, colFamilyName)
* A delete record is return as a ByteArrayPair(recordType, key, null, colFamilyName)
*/
class StateStoreChangelogReaderV2(
fm: CheckpointFileManager,
fileToRead: Path,
compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead,
compressionCodec) {

private def parseBuffer(input: DataInputStream): Array[Byte] = {
val blockSize = input.readInt()
val blockBuffer = new Array[Byte](blockSize)
ByteStreams.readFully(input, blockBuffer, 0, blockSize)
blockBuffer
}

override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = {
val recordTypeSize = input.readInt()
// A -1 key size mean end of file.
if (recordTypeSize == -1) {
finished = true
null
} else if (recordTypeSize < 0) {
throw new IOException(
s"Error reading streaming state file $fileToRead: " +
s"record type size cannot be $recordTypeSize")
} else {
val recordTypeBuffer = new Array[Byte](recordTypeSize)
ByteStreams.readFully(input, recordTypeBuffer, 0, recordTypeSize)
val recordTypeStr = recordTypeBuffer.map(_.toChar).mkString
val recordType = RecordType.withName(recordTypeStr)
recordType match {
case RecordType.PUT_RECORD =>
val keyBuffer = parseBuffer(input)
val valueBuffer = parseBuffer(input)
val colFamilyNameBuffer = parseBuffer(input)
(RecordType.PUT_RECORD, keyBuffer, valueBuffer,
colFamilyNameBuffer.map(_.toChar).mkString)

case RecordType.DELETE_RECORD =>
val keyBuffer = parseBuffer(input)
val valueSize = input.readInt()
assert(valueSize == -1)
val colFamilyNameBuffer = parseBuffer(input)
(RecordType.DELETE_RECORD, keyBuffer, null,
colFamilyNameBuffer.map(_.toChar).mkString)

case _ =>
throw new IOException("Failed to process unknown record type")
}
}
}
Expand Down
Loading

0 comments on commit 16f7c4c

Please sign in to comment.