From 6ee1ad5ac6461446c826cbfd86783a2be1e96f46 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 26 Dec 2024 16:37:10 -0800 Subject: [PATCH 01/37] [SPARK-50655] Move virt col family related mapping into db layer instead of encoder --- .../execution/streaming/state/RocksDB.scala | 66 ++++++++++++------- .../streaming/state/RocksDBFileManager.scala | 29 +++++--- .../streaming/state/RocksDBStateEncoder.scala | 6 +- .../state/RocksDBStateStoreProvider.scala | 12 ++-- .../streaming/state/StateStore.scala | 19 ++---- .../streaming/state/RocksDBSuite.scala | 34 ++++++---- 6 files changed, 101 insertions(+), 65 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 795d29c16bcfb..abaf26f4dddf3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -26,7 +26,7 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import javax.annotation.concurrent.GuardedBy import scala.collection.{mutable, Map} -import scala.jdk.CollectionConverters.{ConcurrentMapHasAsScala, MapHasAsJava} +import scala.jdk.CollectionConverters.ConcurrentMapHasAsScala import scala.ref.WeakReference import scala.util.Try @@ -81,7 +81,8 @@ class RocksDB( checkpointDir: File, version: Long, numKeys: Long, - columnFamilyMapping: Map[String, Short], + numInternalKeys: Long, + columnFamilyMapping: Map[String, ColumnFamilyInfo], maxColumnFamilyId: Short, dfsFileSuffix: String, fileMapping: Map[String, RocksDBSnapshotFile], @@ -179,6 +180,10 @@ class RocksDB( @volatile private var numKeysOnLoadedVersion = 0L @volatile private var numKeysOnWritingVersion = 0L + + @volatile private var numInternalKeysOnLoadedVersion = 0L + @volatile private var numInternalKeysOnWritingVersion = 0L + @volatile private var fileManagerMetrics = RocksDBFileManagerMetrics.EMPTY_METRICS // SPARK-46249 - Keep track of recorded metrics per version which can be used for querying later @@ -192,7 +197,7 @@ class RocksDB( // This is accessed and updated only between load and commit // which means it is implicitly guarded by acquireLock @GuardedBy("acquireLock") - private val colFamilyNameToIdMap = new ConcurrentHashMap[String, Short]() + private val colFamilyNameToInfoMap = new ConcurrentHashMap[String, ColumnFamilyInfo]() @GuardedBy("acquireLock") private val maxColumnFamilyId: AtomicInteger = new AtomicInteger(-1) @@ -210,24 +215,25 @@ class RocksDB( // Methods to fetch column family mapping for this State Store version def getColumnFamilyMapping: Map[String, Short] = { - colFamilyNameToIdMap.asScala + colFamilyNameToInfoMap.asScala.map { case (k, v) => (k, v.cfId) }.toMap } def getColumnFamilyId(cfName: String): Short = { - colFamilyNameToIdMap.get(cfName) + colFamilyNameToInfoMap.get(cfName).cfId } /** * Create RocksDB column family, if not created already */ - def createColFamilyIfAbsent(colFamilyName: String): Short = { + def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean): Short = { if (!checkColFamilyExists(colFamilyName)) { val newColumnFamilyId = maxColumnFamilyId.incrementAndGet().toShort - colFamilyNameToIdMap.putIfAbsent(colFamilyName, newColumnFamilyId) + colFamilyNameToInfoMap.putIfAbsent(colFamilyName, + ColumnFamilyInfo(newColumnFamilyId, isInternal)) shouldForceSnapshot.set(true) newColumnFamilyId } else { - colFamilyNameToIdMap.get(colFamilyName) + colFamilyNameToInfoMap.get(colFamilyName).cfId } } @@ -238,7 +244,7 @@ class RocksDB( def removeColFamilyIfExists(colFamilyName: String): Option[Short] = { if (checkColFamilyExists(colFamilyName)) { shouldForceSnapshot.set(true) - Some(colFamilyNameToIdMap.remove(colFamilyName)) + Some(colFamilyNameToInfoMap.remove(colFamilyName).cfId) } else { None } @@ -251,22 +257,22 @@ class RocksDB( * @return - true if the column family exists, false otherwise */ def checkColFamilyExists(colFamilyName: String): Boolean = { - colFamilyNameToIdMap.containsKey(colFamilyName) + colFamilyNameToInfoMap.containsKey(colFamilyName) } // This method sets the internal column family metadata to // the default values it should be set to on load private def setInitialCFInfo(): Unit = { - colFamilyNameToIdMap.clear() + colFamilyNameToInfoMap.clear() shouldForceSnapshot.set(false) maxColumnFamilyId.set(0) } def getColFamilyCount(isInternal: Boolean): Long = { if (isInternal) { - colFamilyNameToIdMap.asScala.keys.toSeq.count(checkInternalColumnFamilies) + colFamilyNameToInfoMap.asScala.keys.toSeq.count(checkInternalColumnFamilies) } else { - colFamilyNameToIdMap.asScala.keys.toSeq.count(!checkInternalColumnFamilies(_)) + colFamilyNameToInfoMap.asScala.keys.toSeq.count(!checkInternalColumnFamilies(_)) } } @@ -487,7 +493,9 @@ class RocksDB( private def openLocalRocksDB(metadata: RocksDBCheckpointMetadata): Unit = { setInitialCFInfo() metadata.columnFamilyMapping.foreach { mapping => - colFamilyNameToIdMap.putAll(mapping.asJava) + mapping.foreach { case (colFamilyName, colFamilyInfo) => + colFamilyNameToInfoMap.putIfAbsent(colFamilyName, colFamilyInfo) + } } metadata.maxColumnFamilyId.foreach { maxId => @@ -639,7 +647,9 @@ class RocksDB( * Get the value for the given key if present, or null. * @note This will return the last written value even if it was uncommitted. */ - def get(key: Array[Byte]): Array[Byte] = { + def get( + key: Array[Byte], + cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = { db.get(readOptions, key) } @@ -647,7 +657,10 @@ class RocksDB( * Put the given value for the given key. * @note This update is not committed to disk until commit() is called. */ - def put(key: Array[Byte], value: Array[Byte]): Unit = { + def put( + key: Array[Byte], + value: Array[Byte], + cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, key) if (oldValue == null) { @@ -670,7 +683,10 @@ class RocksDB( * * @note This update is not committed to disk until commit() is called. */ - def merge(key: Array[Byte], value: Array[Byte]): Unit = { + def merge( + key: Array[Byte], + value: Array[Byte], + cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, key) if (oldValue == null) { @@ -686,7 +702,7 @@ class RocksDB( * Remove the key if present. * @note This update is not committed to disk until commit() is called. */ - def remove(key: Array[Byte]): Unit = { + def remove(key: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { if (conf.trackTotalNumberOfRows) { val value = db.get(readOptions, key) if (value != null) { @@ -748,7 +764,9 @@ class RocksDB( } } - def prefixScan(prefix: Array[Byte]): Iterator[ByteArrayPair] = { + def prefixScan( + prefix: Array[Byte], + cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = { val iter = db.newIterator() iter.seek(prefix) @@ -824,7 +842,7 @@ class RocksDB( snapshot = Some(createSnapshot( checkpointDir, newVersion, - colFamilyNameToIdMap.asScala.toMap, + colFamilyNameToInfoMap.asScala.toMap, maxColumnFamilyId.get().toShort, sessionStateStoreCkptId)) lastSnapshotVersion = newVersion @@ -1104,14 +1122,15 @@ class RocksDB( private def createSnapshot( checkpointDir: File, version: Long, - columnFamilyMapping: Map[String, Short], + columnFamilyMapping: Map[String, ColumnFamilyInfo], maxColumnFamilyId: Short, uniqueId: Option[String] = None): RocksDBSnapshot = { val (dfsFileSuffix, immutableFileMapping) = rocksDBFileMapping.createSnapshotFileMapping( fileManager, checkpointDir, version) RocksDBSnapshot(checkpointDir, version, numKeysOnWritingVersion, - columnFamilyMapping, maxColumnFamilyId, dfsFileSuffix, immutableFileMapping, uniqueId) + numInternalKeysOnWritingVersion, columnFamilyMapping, maxColumnFamilyId, + dfsFileSuffix, immutableFileMapping, uniqueId) } /** @@ -1232,7 +1251,8 @@ class RocksDB( try { val uploadTime = timeTakenMs { fileManager.saveCheckpointToDfs(snapshot.checkpointDir, - snapshot.version, snapshot.numKeys, snapshot.fileMapping, + snapshot.version, snapshot.numKeys, snapshot.numInternalKeys, + snapshot.fileMapping, Some(snapshot.columnFamilyMapping), Some(snapshot.maxColumnFamilyId), snapshot.uniqueId) fileManagerMetrics = fileManager.latestSaveCheckpointMetrics diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index e42a46dfbe15a..8a20fde3b96b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -248,8 +248,9 @@ class RocksDBFileManager( checkpointDir: File, version: Long, numKeys: Long, + numInternalKeys: Long, fileMapping: Map[String, RocksDBSnapshotFile], - columnFamilyMapping: Option[Map[String, Short]] = None, + columnFamilyMapping: Option[Map[String, ColumnFamilyInfo]] = None, maxColumnFamilyId: Option[Short] = None, checkpointUniqueId: Option[String] = None): Unit = { logFilesInDir(checkpointDir, log"Saving checkpoint files " + @@ -257,8 +258,8 @@ class RocksDBFileManager( val (localImmutableFiles, localOtherFiles) = listRocksDBFiles(checkpointDir) val rocksDBFiles = saveImmutableFilesToDfs( version, localImmutableFiles, fileMapping, checkpointUniqueId) - val metadata = RocksDBCheckpointMetadata(rocksDBFiles, numKeys, columnFamilyMapping, - maxColumnFamilyId) + val metadata = RocksDBCheckpointMetadata(rocksDBFiles, numKeys, numInternalKeys, + columnFamilyMapping, maxColumnFamilyId) val metadataFile = localMetadataFile(checkpointDir) metadata.writeToFile(metadataFile) logInfo(log"Written metadata for version ${MDC(LogKeys.VERSION_NUM, version)}:\n" + @@ -923,6 +924,10 @@ object RocksDBFileManagerMetrics { val EMPTY_METRICS = RocksDBFileManagerMetrics(0L, 0L, 0L, None) } +case class ColumnFamilyInfo( + cfId: Short, + isInternal: Boolean) + /** * Classes to represent metadata of checkpoints saved to DFS. Since this is converted to JSON, any * changes to this MUST be backward-compatible. @@ -931,7 +936,8 @@ case class RocksDBCheckpointMetadata( sstFiles: Seq[RocksDBSstFile], logFiles: Seq[RocksDBLogFile], numKeys: Long, - columnFamilyMapping: Option[Map[String, Short]] = None, + numInternalKeys: Long, + columnFamilyMapping: Option[Map[String, ColumnFamilyInfo]] = None, maxColumnFamilyId: Option[Short] = None) { require(columnFamilyMapping.isDefined == maxColumnFamilyId.isDefined, @@ -997,6 +1003,7 @@ object RocksDBCheckpointMetadata { sstFiles.map(_.asInstanceOf[RocksDBSstFile]), logFiles.map(_.asInstanceOf[RocksDBLogFile]), numKeys, + 0, None, None ) @@ -1005,13 +1012,15 @@ object RocksDBCheckpointMetadata { def apply( rocksDBFiles: Seq[RocksDBImmutableFile], numKeys: Long, - columnFamilyMapping: Option[Map[String, Short]], + numInternalKeys: Long, + columnFamilyMapping: Option[Map[String, ColumnFamilyInfo]], maxColumnFamilyId: Option[Short]): RocksDBCheckpointMetadata = { val (sstFiles, logFiles) = rocksDBFiles.partition(_.isInstanceOf[RocksDBSstFile]) new RocksDBCheckpointMetadata( sstFiles.map(_.asInstanceOf[RocksDBSstFile]), logFiles.map(_.asInstanceOf[RocksDBLogFile]), numKeys, + numInternalKeys, columnFamilyMapping, maxColumnFamilyId ) @@ -1022,20 +1031,22 @@ object RocksDBCheckpointMetadata { sstFiles: Seq[RocksDBSstFile], logFiles: Seq[RocksDBLogFile], numKeys: Long): RocksDBCheckpointMetadata = { - new RocksDBCheckpointMetadata(sstFiles, logFiles, numKeys, None, None) + new RocksDBCheckpointMetadata(sstFiles, logFiles, numKeys, 0, None, None) } // Apply method for cases with column family information def apply( rocksDBFiles: Seq[RocksDBImmutableFile], numKeys: Long, - columnFamilyMapping: Map[String, Short], + numInternalKeys: Long, + columnFamilyMapping: Map[String, ColumnFamilyInfo], maxColumnFamilyId: Short): RocksDBCheckpointMetadata = { val (sstFiles, logFiles) = rocksDBFiles.partition(_.isInstanceOf[RocksDBSstFile]) new RocksDBCheckpointMetadata( sstFiles.map(_.asInstanceOf[RocksDBSstFile]), logFiles.map(_.asInstanceOf[RocksDBLogFile]), numKeys, + numInternalKeys, Some(columnFamilyMapping), Some(maxColumnFamilyId) ) @@ -1046,12 +1057,14 @@ object RocksDBCheckpointMetadata { sstFiles: Seq[RocksDBSstFile], logFiles: Seq[RocksDBLogFile], numKeys: Long, - columnFamilyMapping: Map[String, Short], + numInternalKeys: Long, + columnFamilyMapping: Map[String, ColumnFamilyInfo], maxColumnFamilyId: Short): RocksDBCheckpointMetadata = { new RocksDBCheckpointMetadata( sstFiles, logFiles, numKeys, + numInternalKeys, Some(columnFamilyMapping), Some(maxColumnFamilyId) ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index b4f6197811939..0c842f212b929 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -959,15 +959,13 @@ object RocksDBStateEncoder extends Logging { * @param keyStateEncoderSpec Specification defining the key encoding strategy * (no prefix, prefix scan, or range scan) * @param useColumnFamilies Whether to use RocksDB column families for storage - * @param virtualColFamilyId Optional column family identifier when column families are enabled * @return A configured RocksDBKeyStateEncoder instance */ def getKeyEncoder( dataEncoder: RocksDBDataEncoder, keyStateEncoderSpec: KeyStateEncoderSpec, - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short] = None): RocksDBKeyStateEncoder = { - keyStateEncoderSpec.toEncoder(dataEncoder, useColumnFamilies, virtualColFamilyId) + useColumnFamilies: Boolean): RocksDBKeyStateEncoder = { + keyStateEncoderSpec.toEncoder(dataEncoder, useColumnFamilies) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index fb0bf84d7aabc..9655c0c43e4d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -74,7 +74,7 @@ private[sql] class RocksDBStateStoreProvider useMultipleValuesPerKey: Boolean = false, isInternal: Boolean = false): Unit = { verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) - val newColFamilyId = rocksDB.createColFamilyIfAbsent(colFamilyName) + val newColFamilyId = rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) val dataEncoderCacheKey = StateRowEncoderCacheKey( queryRunId = getRunId(hadoopConf), operatorId = stateStoreId.operatorId, @@ -88,9 +88,7 @@ private[sql] class RocksDBStateStoreProvider val keyEncoder = RocksDBStateEncoder.getKeyEncoder( dataEncoder, keyStateEncoderSpec, - useColumnFamilies, - Some(newColFamilyId) - ) + useColumnFamilies) val valueEncoder = RocksDBStateEncoder.getValueEncoder( dataEncoder, valueSchema, @@ -394,7 +392,8 @@ private[sql] class RocksDBStateStoreProvider var defaultColFamilyId: Option[Short] = None if (useColumnFamilies) { - defaultColFamilyId = Some(rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME)) + defaultColFamilyId = Some(rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, + isInternal = false)) } val dataEncoderCacheKey = StateRowEncoderCacheKey( @@ -410,8 +409,7 @@ private[sql] class RocksDBStateStoreProvider val keyEncoder = RocksDBStateEncoder.getKeyEncoder( dataEncoder, keyStateEncoderSpec, - useColumnFamilies, - defaultColFamilyId + useColumnFamilies ) val valueEncoder = RocksDBStateEncoder.getValueEncoder( dataEncoder, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index de10518035e2f..b0a511cd0e902 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -331,13 +331,11 @@ sealed trait KeyStateEncoderSpec { * * @param dataEncoder The encoder to handle the actual data encoding/decoding * @param useColumnFamilies Whether to use RocksDB column families - * @param virtualColFamilyId Optional column family ID when column families are used * @return A RocksDBKeyStateEncoder configured for this spec */ def toEncoder( dataEncoder: RocksDBDataEncoder, - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder + useColumnFamilies: Boolean): RocksDBKeyStateEncoder } object KeyStateEncoderSpec { @@ -364,10 +362,9 @@ case class NoPrefixKeyStateEncoderSpec(keySchema: StructType) extends KeyStateEn override def toEncoder( dataEncoder: RocksDBDataEncoder, - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder = { + useColumnFamilies: Boolean): RocksDBKeyStateEncoder = { new NoPrefixKeyStateEncoder( - dataEncoder, keySchema, useColumnFamilies, virtualColFamilyId) + dataEncoder, keySchema, useColumnFamilies) } } @@ -380,10 +377,9 @@ case class PrefixKeyScanStateEncoderSpec( override def toEncoder( dataEncoder: RocksDBDataEncoder, - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder = { + useColumnFamilies: Boolean): RocksDBKeyStateEncoder = { new PrefixKeyScanStateEncoder( - dataEncoder, keySchema, numColsPrefixKey, useColumnFamilies, virtualColFamilyId) + dataEncoder, keySchema, numColsPrefixKey, useColumnFamilies) } @@ -403,10 +399,9 @@ case class RangeKeyScanStateEncoderSpec( override def toEncoder( dataEncoder: RocksDBDataEncoder, - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short]): RocksDBKeyStateEncoder = { + useColumnFamilies: Boolean): RocksDBKeyStateEncoder = { new RangeKeyScanStateEncoder( - dataEncoder, keySchema, orderingOrdinals, useColumnFamilies, virtualColFamilyId) + dataEncoder, keySchema, orderingOrdinals, useColumnFamilies) } override def jsonValue: JValue = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 3ec81a7ef13e4..c5adbdfccbea5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -1244,7 +1244,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession val conf = dbConf.copy(minDeltasForSnapshot = 5, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created withDB(remoteDir, conf = conf, useColumnFamilies = true) { db => - db.createColFamilyIfAbsent("test") + db.createColFamilyIfAbsent("test", isInternal = false) db.load(0) db.put("a", "1") db.put("b", "2") @@ -1345,7 +1345,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession case true => Some(UUID.randomUUID().toString) } saveCheckpointFiles(fileManager, cpFiles1, version = 1, - numKeys = 101, rocksDBFileMapping, uuid) + numKeys = 101, rocksDBFileMapping, + numInternalKeys = 0, uuid) assert(fileManager.getLatestVersion() === 1) assert(numRemoteSSTFiles == 2) // 2 sst files copied assert(numRemoteLogFiles == 2) @@ -1360,7 +1361,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession "archive/00003.log" -> 2000 ) saveCheckpointFiles(fileManager_, cpFiles1_, version = 1, - numKeys = 101, new RocksDBFileMapping(), uuid) + numKeys = 101, new RocksDBFileMapping(), + numInternalKeys = 0, uuid) assert(fileManager_.getLatestVersion() === 1) assert(numRemoteSSTFiles == 4) assert(numRemoteLogFiles == 4) @@ -1380,7 +1382,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession "archive/00005.log" -> 2000 ) saveCheckpointFiles(fileManager_, cpFiles2, - version = 2, numKeys = 121, new RocksDBFileMapping(), uuid) + version = 2, numKeys = 121, new RocksDBFileMapping(), + numInternalKeys = 0, uuid) fileManager_.deleteOldVersions(1) assert(numRemoteSSTFiles <= 4) // delete files recorded in 1.zip assert(numRemoteLogFiles <= 5) // delete files recorded in 1.zip and orphan 00001.log @@ -1395,7 +1398,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession "archive/00007.log" -> 2000 ) saveCheckpointFiles(fileManager_, cpFiles3, - version = 3, numKeys = 131, new RocksDBFileMapping(), uuid) + version = 3, numKeys = 131, new RocksDBFileMapping(), + numInternalKeys = 0, uuid) assert(fileManager_.getLatestVersion() === 3) fileManager_.deleteOldVersions(1) assert(numRemoteSSTFiles == 1) @@ -1441,7 +1445,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } saveCheckpointFiles( - fileManager, cpFiles1, version = 1, numKeys = 101, rocksDBFileMapping, uuid) + fileManager, cpFiles1, version = 1, numKeys = 101, rocksDBFileMapping, + numInternalKeys = 0, uuid) fileManager.deleteOldVersions(1) // Should not delete orphan files even when they are older than all existing files // when there is only 1 version. @@ -1459,7 +1464,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession "archive/00004.log" -> 2000 ) saveCheckpointFiles( - fileManager, cpFiles2, version = 2, numKeys = 101, rocksDBFileMapping, uuid) + fileManager, cpFiles2, version = 2, numKeys = 101, rocksDBFileMapping, + numInternalKeys = 0, uuid) assert(numRemoteSSTFiles == 5) assert(numRemoteLogFiles == 5) fileManager.deleteOldVersions(1) @@ -1509,7 +1515,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } saveCheckpointFiles( - fileManager, cpFiles1, version = 1, numKeys = 101, fileMapping, uuid) + fileManager, cpFiles1, version = 1, numKeys = 101, fileMapping, numInternalKeys = 0, + uuid) assert(fileManager.getLatestVersion() === 1) assert(numRemoteSSTFiles == 2) // 2 sst files copied assert(numRemoteLogFiles == 2) // 2 log files copied @@ -1547,7 +1554,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // upload version 1 again, new checkpoint will be created and SST files from // previously committed version 1 will not be reused. saveCheckpointFiles(fileManager, cpFiles1_, - version = 1, numKeys = 1001, fileMapping, uuid) + version = 1, numKeys = 1001, fileMapping, + numInternalKeys = 0, uuid) assert(numRemoteSSTFiles === 5, "shouldn't reuse old version 1 SST files" + " while uploading version 1 again") // 2 old + 3 new SST files assert(numRemoteLogFiles === 5, "shouldn't reuse old version 1 log files" + @@ -1567,7 +1575,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession "archive/00004.log" -> 4000 ) saveCheckpointFiles(fileManager, cpFiles2, - version = 2, numKeys = 1501, fileMapping, uuid) + version = 2, numKeys = 1501, fileMapping, + numInternalKeys = 0, uuid) assert(numRemoteSSTFiles === 6) // 1 new file over earlier 5 files assert(numRemoteLogFiles === 6) // 1 new file over earlier 6 files loadAndVerifyCheckpointFiles(fileManager, verificationDir, @@ -1610,7 +1619,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } intercept[IOException] { saveCheckpointFiles( - fileManager, cpFiles, version = 1, numKeys = 101, new RocksDBFileMapping(), uuid) + fileManager, cpFiles, version = 1, numKeys = 101, new RocksDBFileMapping(), + numInternalKeys = 0, uuid) } assert(CreateAtomicTestManager.cancelCalledInCreateAtomic) } @@ -3013,6 +3023,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession version: Int, numKeys: Int, fileMapping: RocksDBFileMapping, + numInternalKeys: Int = 0, checkpointUniqueId: Option[String] = None): Unit = { val checkpointDir = Utils.createTempDir().getAbsolutePath // local dir to create checkpoints generateFiles(checkpointDir, fileToLengths) @@ -3022,6 +3033,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession checkpointDir, version, numKeys, + numInternalKeys, immutableFileMapping, checkpointUniqueId = checkpointUniqueId) From 1eea64ab673c692a31de563fa38e4a170a6288ea Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 26 Dec 2024 18:39:22 -0800 Subject: [PATCH 02/37] Misc fix --- .../execution/streaming/state/RocksDB.scala | 86 +++++++++++++++---- .../streaming/state/RocksDBStateEncoder.scala | 57 +++--------- .../state/RocksDBStateStoreProvider.scala | 52 +++-------- .../streaming/state/StateStore.scala | 2 + .../state/RocksDBStateStoreSuite.scala | 4 - 5 files changed, 96 insertions(+), 105 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index abaf26f4dddf3..7fedf19a141cc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -41,6 +41,7 @@ import org.apache.spark.TaskContext import org.apache.spark.internal.{LogEntry, Logging, LogKeys, MDC} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.unsafe.Platform import org.apache.spark.util.{NextIterator, Utils} // RocksDB operations that could acquire/release the instance lock @@ -199,6 +200,9 @@ class RocksDB( @GuardedBy("acquireLock") private val colFamilyNameToInfoMap = new ConcurrentHashMap[String, ColumnFamilyInfo]() + @GuardedBy("acquireLock") + private val colFamilyIdToNameMap = new ConcurrentHashMap[Short, String]() + @GuardedBy("acquireLock") private val maxColumnFamilyId: AtomicInteger = new AtomicInteger(-1) @@ -218,8 +222,8 @@ class RocksDB( colFamilyNameToInfoMap.asScala.map { case (k, v) => (k, v.cfId) }.toMap } - def getColumnFamilyId(cfName: String): Short = { - colFamilyNameToInfoMap.get(cfName).cfId + def getColumnFamilyInfo(cfName: String): ColumnFamilyInfo = { + colFamilyNameToInfoMap.get(cfName) } /** @@ -230,6 +234,7 @@ class RocksDB( val newColumnFamilyId = maxColumnFamilyId.incrementAndGet().toShort colFamilyNameToInfoMap.putIfAbsent(colFamilyName, ColumnFamilyInfo(newColumnFamilyId, isInternal)) + colFamilyIdToNameMap.putIfAbsent(newColumnFamilyId, colFamilyName) shouldForceSnapshot.set(true) newColumnFamilyId } else { @@ -241,12 +246,15 @@ class RocksDB( * Remove RocksDB column family, if exists * @return columnFamilyId if it exists, else None */ - def removeColFamilyIfExists(colFamilyName: String): Option[Short] = { + def removeColFamilyIfExists(colFamilyName: String): Boolean = { if (checkColFamilyExists(colFamilyName)) { shouldForceSnapshot.set(true) - Some(colFamilyNameToInfoMap.remove(colFamilyName).cfId) + val colFamilyInfo = colFamilyNameToInfoMap.get(colFamilyName) + colFamilyNameToInfoMap.remove(colFamilyName) + colFamilyIdToNameMap.remove(colFamilyInfo.cfId) + true } else { - None + false } } @@ -264,6 +272,7 @@ class RocksDB( // the default values it should be set to on load private def setInitialCFInfo(): Unit = { colFamilyNameToInfoMap.clear() + colFamilyIdToNameMap.clear() shouldForceSnapshot.set(false) maxColumnFamilyId.set(0) } @@ -495,6 +504,7 @@ class RocksDB( metadata.columnFamilyMapping.foreach { mapping => mapping.foreach { case (colFamilyName, colFamilyInfo) => colFamilyNameToInfoMap.putIfAbsent(colFamilyName, colFamilyInfo) + colFamilyIdToNameMap.putIfAbsent(colFamilyInfo.cfId, colFamilyName) } } @@ -625,16 +635,34 @@ class RocksDB( var changelogReader: StateStoreChangelogReader = null try { changelogReader = fileManager.getChangelogReader(v, uniqueId) - changelogReader.foreach { case (recordType, key, value) => - recordType match { - case RecordType.PUT_RECORD => - put(key, value) - case RecordType.DELETE_RECORD => - remove(key) + if (useColumnFamilies) { + changelogReader.foreach { case (recordType, key, value) => + val cfId = Platform.getShort(key, Platform.BYTE_ARRAY_OFFSET) + val cfName = colFamilyIdToNameMap.get(cfId) + recordType match { + case RecordType.PUT_RECORD => + put(key, value, cfName) + + case RecordType.DELETE_RECORD => + remove(key, cfName) + + case RecordType.MERGE_RECORD => + merge(key, value, cfName) + } + } + } else { + changelogReader.foreach { case (recordType, key, value) => + recordType match { + case RecordType.PUT_RECORD => + put(key, value) + + case RecordType.DELETE_RECORD => + remove(key) - case RecordType.MERGE_RECORD => - merge(key, value) + case RecordType.MERGE_RECORD => + merge(key, value) + } } } } finally { @@ -650,6 +678,11 @@ class RocksDB( def get( key: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = { + if (useColumnFamilies) { + val cfInfo = getColumnFamilyInfo(cfName) + Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + } + db.get(readOptions, key) } @@ -661,6 +694,11 @@ class RocksDB( key: Array[Byte], value: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + if (useColumnFamilies) { + val cfInfo = getColumnFamilyInfo(cfName) + Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + } + if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, key) if (oldValue == null) { @@ -687,6 +725,11 @@ class RocksDB( key: Array[Byte], value: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + if (useColumnFamilies) { + val cfInfo = getColumnFamilyInfo(cfName) + Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + } + if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, key) if (oldValue == null) { @@ -703,6 +746,11 @@ class RocksDB( * @note This update is not committed to disk until commit() is called. */ def remove(key: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + if (useColumnFamilies) { + val cfInfo = getColumnFamilyInfo(cfName) + Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + } + if (conf.trackTotalNumberOfRows) { val value = db.get(readOptions, key) if (value != null) { @@ -764,9 +812,7 @@ class RocksDB( } } - def prefixScan( - prefix: Array[Byte], - cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = { + def prefixScan(prefix: Array[Byte]): Iterator[ByteArrayPair] = { val iter = db.newIterator() iter.seek(prefix) @@ -792,6 +838,14 @@ class RocksDB( } } + def prefixScan(cfName: String): Iterator[ByteArrayPair] = { + assert(useColumnFamilies == true) + val cfBytes = new Array[Byte](StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) + val cfInfo = getColumnFamilyInfo(cfName) + Platform.putShort(cfBytes, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + prefixScan(cfBytes) + } + /** * Commit all the updates made as a version to DFS. The steps it needs to do to commits are: * - Flush all changes to disk diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index 0c842f212b929..18196608e3858 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{BoundReference, JoinedRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter import org.apache.spark.sql.execution.streaming.StateStoreColumnFamilySchemaUtils -import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider.{STATE_ENCODING_NUM_VERSION_BYTES, STATE_ENCODING_VERSION, VIRTUAL_COL_FAMILY_PREFIX_BYTES} +import org.apache.spark.sql.execution.streaming.state.RocksDBStateStoreProvider.{STATE_ENCODING_NUM_VERSION_BYTES, STATE_ENCODING_VERSION} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform @@ -41,7 +41,6 @@ sealed trait RocksDBKeyStateEncoder { def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] def encodeKey(row: UnsafeRow): Array[Byte] def decodeKey(keyBytes: Array[Byte]): UnsafeRow - def getColumnFamilyIdBytes(): Array[Byte] } sealed trait RocksDBValueStateEncoder { @@ -895,26 +894,13 @@ class AvroStateEncoder( bytes, avroEncoder.valueDeserializer, valueAvroType, valueProj) } -abstract class RocksDBKeyStateEncoderBase( - useColumnFamilies: Boolean, - virtualColFamilyId: Option[Short] = None) extends RocksDBKeyStateEncoder { +abstract class RocksDBKeyStateEncoderBase(useColumnFamilies: Boolean) + extends RocksDBKeyStateEncoder { def offsetForColFamilyPrefix: Int = - if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + if (useColumnFamilies) StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val out = new ByteArrayOutputStream - /** - * Get Byte Array for the virtual column family id that is used as prefix for - * key state rows. - */ - override def getColumnFamilyIdBytes(): Array[Byte] = { - assert(useColumnFamilies, "Cannot return virtual Column Family Id Bytes" + - " because multiple Column is not supported for this encoder") - val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId.get) - encodedBytes - } - /** * Encode and put column family Id as a prefix to a pre-allocated byte array. * @@ -927,7 +913,6 @@ abstract class RocksDBKeyStateEncoderBase( val encodedBytes = new Array[Byte](numBytes + offsetForColFamilyPrefix) var offset = Platform.BYTE_ARRAY_OFFSET if (useColumnFamilies) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId.get) offset = Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix } (encodedBytes, offset) @@ -938,7 +923,7 @@ abstract class RocksDBKeyStateEncoderBase( */ protected def decodeKeyStartOffset: Int = { if (useColumnFamilies) { - Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES + Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES } else Platform.BYTE_ARRAY_OFFSET } } @@ -987,21 +972,6 @@ object RocksDBStateEncoder extends Logging { new SingleValueStateEncoder(dataEncoder, valueSchema) } } - - /** - * Encodes a virtual column family ID into a byte array suitable for RocksDB. - * - * This method creates a fixed-size byte array prefixed with the virtual column family ID, - * which is used to partition data within RocksDB. - * - * @param virtualColFamilyId The column family identifier to encode - * @return A byte array containing the encoded column family ID - */ - def getColumnFamilyIdBytes(virtualColFamilyId: Short): Array[Byte] = { - val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId) - encodedBytes - } } /** @@ -1016,9 +986,8 @@ class PrefixKeyScanStateEncoder( dataEncoder: RocksDBDataEncoder, keySchema: StructType, numColsPrefixKey: Int, - useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None) - extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { + useColumnFamilies: Boolean = false) + extends RocksDBKeyStateEncoderBase(useColumnFamilies) with Logging { private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { keySchema.zipWithIndex.take(numColsPrefixKey) @@ -1141,9 +1110,8 @@ class RangeKeyScanStateEncoder( dataEncoder: RocksDBDataEncoder, keySchema: StructType, orderingOrdinals: Seq[Int], - useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None) - extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { + useColumnFamilies: Boolean = false) + extends RocksDBKeyStateEncoderBase(useColumnFamilies) with Logging { private val rangeScanKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { orderingOrdinals.map { ordinal => @@ -1313,9 +1281,8 @@ class RangeKeyScanStateEncoder( class NoPrefixKeyStateEncoder( dataEncoder: RocksDBDataEncoder, keySchema: StructType, - useColumnFamilies: Boolean = false, - virtualColFamilyId: Option[Short] = None) - extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) with Logging { + useColumnFamilies: Boolean = false) + extends RocksDBKeyStateEncoderBase(useColumnFamilies) with Logging { override def encodeKey(row: UnsafeRow): Array[Byte] = { if (!useColumnFamilies) { @@ -1346,7 +1313,7 @@ class NoPrefixKeyStateEncoder( if (keyBytes != null) { // Create new byte array without prefix val dataLength = keyBytes.length - - STATE_ENCODING_NUM_VERSION_BYTES - VIRTUAL_COL_FAMILY_PREFIX_BYTES + STATE_ENCODING_NUM_VERSION_BYTES - StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES val dataBytes = new Array[Byte](dataLength) Platform.copyMemory( keyBytes, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 9655c0c43e4d0..4adec4233515c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -62,10 +62,6 @@ private[sql] class RocksDBStateStoreProvider rocksDB.getColumnFamilyMapping.toMap } - private[sql] def getColumnFamilyId(cfName: String): Short = { - rocksDB.getColumnFamilyId(cfName) - } - override def createColFamilyIfAbsent( colFamilyName: String, keySchema: StructType, @@ -74,7 +70,7 @@ private[sql] class RocksDBStateStoreProvider useMultipleValuesPerKey: Boolean = false, isInternal: Boolean = false): Unit = { verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) - val newColFamilyId = rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) + rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) val dataEncoderCacheKey = StateRowEncoderCacheKey( queryRunId = getRunId(hadoopConf), operatorId = stateStoreId.operatorId, @@ -185,7 +181,7 @@ private[sql] class RocksDBStateStoreProvider // we'll need to do prefixScan on the default column family with the same column // family id prefix to get all rows stored in a given virtual column family if (useColumnFamilies) { - rocksDB.prefixScan(kvEncoder._1.getColumnFamilyIdBytes()).map { kv => + rocksDB.prefixScan(colFamilyName).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) if (!isValidated && rowPair.value != null && !useColumnFamilies) { @@ -342,21 +338,14 @@ private[sql] class RocksDBStateStoreProvider verifyColFamilyCreationOrDeletion("remove_col_family", colFamilyName) verify(useColumnFamilies, "Column families are not supported in this store") - val result = { - val colFamilyId = rocksDB.removeColFamilyIfExists(colFamilyName) - - colFamilyId match { - case Some(vcfId) => - val colFamilyIdBytes = - RocksDBStateEncoder.getColumnFamilyIdBytes(vcfId) - rocksDB.prefixScan(colFamilyIdBytes).foreach { kv => - rocksDB.remove(kv.key) - } - true - case None => false + val result = rocksDB.removeColFamilyIfExists(colFamilyName) + + if (result) { + rocksDB.prefixScan(colFamilyName).foreach { kv => + rocksDB.remove(kv.key) } + keyValueEncoderMap.remove(colFamilyName) } - keyValueEncoderMap.remove(colFamilyName) result } } @@ -389,11 +378,9 @@ private[sql] class RocksDBStateStoreProvider } rocksDB // lazy initialization - var defaultColFamilyId: Option[Short] = None if (useColumnFamilies) { - defaultColFamilyId = Some(rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, - isInternal = false)) + rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) } val dataEncoderCacheKey = StateRowEncoderCacheKey( @@ -637,7 +624,6 @@ object RocksDBStateStoreProvider { // Version as a single byte that specifies the encoding of the row data in RocksDB val STATE_ENCODING_NUM_VERSION_BYTES = 1 val STATE_ENCODING_VERSION: Byte = 0 - val VIRTUAL_COL_FAMILY_PREFIX_BYTES = 2 private val MAX_AVRO_ENCODERS_IN_CACHE = 1000 private val AVRO_ENCODER_LIFETIME_HOURS = 1L @@ -801,28 +787,13 @@ class RocksDBStateStoreChangeDataReader( override protected var changelogSuffix: String = "changelog" - private def getColFamilyIdBytes: Option[Array[Byte]] = { - if (colFamilyNameOpt.isDefined) { - val colFamilyName = colFamilyNameOpt.get - if (!keyValueEncoderMap.containsKey(colFamilyName)) { - throw new IllegalStateException( - s"Column family $colFamilyName not found in the key value encoder map") - } - Some(keyValueEncoderMap.get(colFamilyName)._1.getColumnFamilyIdBytes()) - } else { - None - } - } - - private val colFamilyIdBytesOpt: Option[Array[Byte]] = getColFamilyIdBytes - override def getNext(): (RecordType.Value, UnsafeRow, UnsafeRow, Long) = { var currRecord: (RecordType.Value, Array[Byte], Array[Byte]) = null val currEncoder: (RocksDBKeyStateEncoder, RocksDBValueStateEncoder) = keyValueEncoderMap.get(colFamilyNameOpt .getOrElse(StateStore.DEFAULT_COL_FAMILY_NAME)) - if (colFamilyIdBytesOpt.isDefined) { + if (colFamilyNameOpt.isDefined) { // If we are reading records for a particular column family, the corresponding vcf id // will be encoded in the key byte array. We need to extract that and compare for the // expected column family id. If it matches, we return the record. If not, we move to @@ -834,8 +805,9 @@ class RocksDBStateStoreChangeDataReader( return null } + // TODO: fix this val nextRecord = reader.next() - val colFamilyIdBytes: Array[Byte] = colFamilyIdBytesOpt.get + val colFamilyIdBytes: Array[Byte] = new Array[Byte](2) val endIndex = colFamilyIdBytes.size // Function checks for byte arrays being equal // from index 0 to endIndex - 1 (both inclusive) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index b0a511cd0e902..57dbe895ce435 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -695,6 +695,8 @@ object StateStore extends Logging { val DEFAULT_COL_FAMILY_NAME = "default" + val VIRTUAL_COL_FAMILY_PREFIX_BYTES = 2 + @GuardedBy("loadedProviders") private val loadedProviders = new mutable.HashMap[StateStoreProviderId, StateStoreProvider]() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala index a8e9d6db01e24..40307252f7ecc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala @@ -1122,7 +1122,6 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid store = provider.getRocksDBStateStore(2) store.createColFamilyIfAbsent(colFamily3, keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) - assert(store.getColumnFamilyId(colFamily3) == 3) store.removeColFamilyIfExists(colFamily1) store.removeColFamilyIfExists(colFamily3) store.commit() @@ -1131,15 +1130,12 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid // this should return the old id, because we didn't remove this colFamily for version 1 store.createColFamilyIfAbsent(colFamily1, keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) - assert(store.getColumnFamilyId(colFamily1) == 1) store = provider.getRocksDBStateStore(3) store.createColFamilyIfAbsent(colFamily4, keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) - assert(store.getColumnFamilyId(colFamily4) == 4) store.createColFamilyIfAbsent(colFamily5, keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) - assert(store.getColumnFamilyId(colFamily5) == 5) } } From 3f341fb96b6f4f4adb8e2384c506b5906becc548 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 26 Dec 2024 18:50:25 -0800 Subject: [PATCH 03/37] Misc change --- .../spark/sql/execution/streaming/state/RocksDB.scala | 7 +------ .../streaming/state/RocksDBStateStoreProvider.scala | 7 ------- 2 files changed, 1 insertion(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 7fedf19a141cc..6223cf4cfa3f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -217,12 +217,7 @@ class RocksDB( */ private def checkInternalColumnFamilies(cfName: String): Boolean = cfName.charAt(0) == '_' - // Methods to fetch column family mapping for this State Store version - def getColumnFamilyMapping: Map[String, Short] = { - colFamilyNameToInfoMap.asScala.map { case (k, v) => (k, v.cfId) }.toMap - } - - def getColumnFamilyInfo(cfName: String): ColumnFamilyInfo = { + private def getColumnFamilyInfo(cfName: String): ColumnFamilyInfo = { colFamilyNameToInfoMap.get(cfName) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 4adec4233515c..631e28c63a194 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -55,13 +55,6 @@ private[sql] class RocksDBStateStoreProvider override def version: Long = lastVersion - // Test-visible methods to fetch column family mapping for this State Store version - // Because column families are only enabled for RocksDBStateStore, these methods - // are no-ops everywhere else. - private[sql] def getColumnFamilyMapping: Map[String, Short] = { - rocksDB.getColumnFamilyMapping.toMap - } - override def createColFamilyIfAbsent( colFamilyName: String, keySchema: StructType, From 30c2bd709b340d7cf55633f8dbd98ec0785dbde8 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 27 Dec 2024 10:34:36 -0800 Subject: [PATCH 04/37] Misc fix --- .../streaming/state/RocksDBStateEncoder.scala | 16 ---------------- 1 file changed, 16 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index 89d6caf613b20..652940d256d38 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -1051,22 +1051,6 @@ class StateRowPrefixEncoder(useColumnFamilies: Boolean) { result } -/* - def decodeStateRowPrefix(stateRow: Array[Byte]): StateRowPrefix = { - var offset = Platform.BYTE_ARRAY_OFFSET - - // Read column family ID if present - val colFamilyId = if (useColumnFamilies) { - val id = Platform.getShort(stateRow, offset) - offset += StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES - Some(id) - } else { - None - } - - StateRowPrefix(colFamilyId) - } */ - def decodeStateRowData(stateRow: Array[Byte]): Array[Byte] = { val offset = Platform.BYTE_ARRAY_OFFSET + getNumPrefixBytes From 1c07d73dc066940b129bc0941d879aa6ac5e7ee2 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 27 Dec 2024 11:40:10 -0800 Subject: [PATCH 05/37] Misc fix --- .../execution/streaming/state/RocksDB.scala | 129 +++++++++++++----- 1 file changed, 97 insertions(+), 32 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 2ac946a3e1597..71e32cc4d43c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -384,6 +384,7 @@ class RocksDB( // After changelog replay the numKeysOnWritingVersion will be updated to // the correct number of keys in the loaded version. numKeysOnLoadedVersion = numKeysOnWritingVersion + numInternalKeysOnLoadedVersion = numInternalKeysOnWritingVersion fileManagerMetrics = fileManager.latestLoadCheckpointMetrics } @@ -457,6 +458,7 @@ class RocksDB( // After changelog replay the numKeysOnWritingVersion will be updated to // the correct number of keys in the loaded version. numKeysOnLoadedVersion = numKeysOnWritingVersion + numInternalKeysOnLoadedVersion = numInternalKeysOnWritingVersion fileManagerMetrics = fileManager.latestLoadCheckpointMetrics } if (conf.resetStatsOnLoad) { @@ -492,16 +494,20 @@ class RocksDB( maxColumnFamilyId.set(maxId) } openDB() - numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) { - // we don't track the total number of rows - discard the number being track - -1L - } else if (metadata.numKeys < 0) { - // we track the total number of rows, but the snapshot doesn't have tracking number - // need to count keys now - countKeys() - } else { - metadata.numKeys + val (numKeys, numInternalKeys) = { + if (!conf.trackTotalNumberOfRows) { + // we don't track the total number of rows - discard the number being track + (-1L, -1L) + } else if (metadata.numKeys < 0) { + // we track the total number of rows, but the snapshot doesn't have tracking number + // need to count keys now + countKeys() + } else { + (metadata.numKeys, metadata.numInternalKeys) + } } + numKeysOnWritingVersion = numKeys + numInternalKeysOnWritingVersion = numInternalKeys } def load( @@ -569,16 +575,19 @@ class RocksDB( lastSnapshotVersion = snapshotVersion openDB() - numKeysOnWritingVersion = if (!conf.trackTotalNumberOfRows) { + val (numKeys, numInternalKeys) = if (!conf.trackTotalNumberOfRows) { // we don't track the total number of rows - discard the number being track - -1L + (-1L, -1L) } else if (metadata.numKeys < 0) { // we track the total number of rows, but the snapshot doesn't have tracking number // need to count keys now countKeys() } else { - metadata.numKeys + (metadata.numKeys, metadata.numInternalKeys) } + numKeysOnWritingVersion = numKeys + numInternalKeysOnWritingVersion = numInternalKeys + if (loadedVersion != endVersion) { val versionsAndUniqueIds: Array[(Long, Option[String])] = (loadedVersion + 1 to endVersion).map((_, None)).toArray @@ -588,6 +597,7 @@ class RocksDB( // After changelog replay the numKeysOnWritingVersion will be updated to // the correct number of keys in the loaded version. numKeysOnLoadedVersion = numKeysOnWritingVersion + numInternalKeysOnLoadedVersion = numInternalKeysOnWritingVersion fileManagerMetrics = fileManager.latestLoadCheckpointMetrics if (conf.resetStatsOnLoad) { @@ -677,12 +687,23 @@ class RocksDB( if (useColumnFamilies) { val cfInfo = getColumnFamilyInfo(cfName) Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) - } - if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, key) - if (oldValue == null) { - numKeysOnWritingVersion += 1 + if (conf.trackTotalNumberOfRows) { + val oldValue = db.get(readOptions, key) + if (oldValue == null) { + if (cfInfo.isInternal) { + numInternalKeysOnWritingVersion += 1 + } else { + numKeysOnWritingVersion += 1 + } + } + } + } else { + if (conf.trackTotalNumberOfRows) { + val oldValue = db.get(readOptions, key) + if (oldValue == null) { + numKeysOnWritingVersion += 1 + } } } @@ -708,14 +729,26 @@ class RocksDB( if (useColumnFamilies) { val cfInfo = getColumnFamilyInfo(cfName) Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) - } - if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, key) - if (oldValue == null) { - numKeysOnWritingVersion += 1 + if (conf.trackTotalNumberOfRows) { + val oldValue = db.get(readOptions, key) + if (oldValue == null) { + if (cfInfo.isInternal) { + numInternalKeysOnWritingVersion += 1 + } else { + numKeysOnWritingVersion += 1 + } + } + } + } else { + if (conf.trackTotalNumberOfRows) { + val oldValue = db.get(readOptions, key) + if (oldValue == null) { + numKeysOnWritingVersion += 1 + } } } + db.merge(writeOptions, key, value) changelogWriter.foreach(_.merge(key, value)) @@ -729,14 +762,26 @@ class RocksDB( if (useColumnFamilies) { val cfInfo = getColumnFamilyInfo(cfName) Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) - } - if (conf.trackTotalNumberOfRows) { - val value = db.get(readOptions, key) - if (value != null) { - numKeysOnWritingVersion -= 1 + if (conf.trackTotalNumberOfRows) { + val oldValue = db.get(readOptions, key) + if (oldValue == null) { + if (cfInfo.isInternal) { + numInternalKeysOnWritingVersion -= 1 + } else { + numKeysOnWritingVersion -= 1 + } + } + } + } else { + if (conf.trackTotalNumberOfRows) { + val value = db.get(readOptions, key) + if (value != null) { + numKeysOnWritingVersion -= 1 + } } } + db.delete(writeOptions, key) changelogWriter.foreach(_.delete(key)) } @@ -771,7 +816,7 @@ class RocksDB( } } - private def countKeys(): Long = { + private def countKeys(): (Long, Long) = { val iter = db.newIterator() try { @@ -781,12 +826,28 @@ class RocksDB( iter.seekToFirst() var keys = 0L - while (iter.isValid) { - keys += 1 - iter.next() + var internalKeys = 0L + + if (!useColumnFamilies) { + while (iter.isValid) { + keys += 1 + iter.next() + } + } else { + while (iter.isValid) { + val cfId = Platform.getShort(iter.key, Platform.BYTE_ARRAY_OFFSET) + val cfName = colFamilyIdToNameMap.get(cfId) + val cfInfo = getColumnFamilyInfo(cfName) + if (cfInfo.isInternal) { + internalKeys += 1 + } else { + keys += 1 + } + iter.next() + } } - keys + (keys, internalKeys) } finally { iter.close() } @@ -893,6 +954,7 @@ class RocksDB( fileManager.setMaxSeenVersion(newVersion) numKeysOnLoadedVersion = numKeysOnWritingVersion + numInternalKeysOnLoadedVersion = numInternalKeysOnWritingVersion loadedVersion = newVersion commitLatencyMs ++= Map( "fileSync" -> fileSyncTimeMs @@ -991,6 +1053,7 @@ class RocksDB( acquire(RollbackStore) try { numKeysOnWritingVersion = numKeysOnLoadedVersion + numInternalKeysOnWritingVersion = numInternalKeysOnLoadedVersion loadedVersion = -1L lastCommitBasedStateStoreCkptId = None lastCommittedStateStoreCkptId = None @@ -1131,6 +1194,7 @@ class RocksDB( RocksDBMetrics( numKeysOnLoadedVersion, numKeysOnWritingVersion, + numInternalKeysOnWritingVersion, memoryUsage, pinnedBlocksMemUsage, totalSSTFilesBytes, @@ -1748,6 +1812,7 @@ object RocksDBConf { case class RocksDBMetrics( numCommittedKeys: Long, numUncommittedKeys: Long, + numInternalKeys: Long, totalMemUsageBytes: Long, pinnedBlocksMemUsage: Long, totalSSTFilesBytes: Long, From a156092dbe259b6bdb77fcde25dbbc1d997ec969 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 27 Dec 2024 15:54:26 -0800 Subject: [PATCH 06/37] Misc fix --- .../execution/streaming/state/RocksDB.scala | 132 ++++++++++++------ .../streaming/state/RocksDBStateEncoder.scala | 86 ++---------- .../state/RocksDBStateStoreProvider.scala | 6 +- .../streaming/state/RocksDBSuite.scala | 89 +++++++++++- 4 files changed, 191 insertions(+), 122 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 71e32cc4d43c0..b7632184c7b04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -211,6 +211,7 @@ class RocksDB( */ def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean): Short = { if (!checkColFamilyExists(colFamilyName)) { + logWarning(s"TEST: creating column family $colFamilyName") val newColumnFamilyId = maxColumnFamilyId.incrementAndGet().toShort colFamilyNameToInfoMap.putIfAbsent(colFamilyName, ColumnFamilyInfo(newColumnFamilyId, isInternal)) @@ -628,17 +629,16 @@ class RocksDB( if (useColumnFamilies) { changelogReader.foreach { case (recordType, key, value) => - val cfId = Platform.getShort(key, Platform.BYTE_ARRAY_OFFSET) - val cfName = colFamilyIdToNameMap.get(cfId) + val (keyWithoutPrefix, cfName) = decodeStateRowWithPrefix(key) recordType match { case RecordType.PUT_RECORD => - put(key, value, cfName) + put(keyWithoutPrefix, value, cfName) case RecordType.DELETE_RECORD => - remove(key, cfName) + remove(keyWithoutPrefix, cfName) case RecordType.MERGE_RECORD => - merge(key, value, cfName) + merge(keyWithoutPrefix, value, cfName) } } } else { @@ -661,6 +661,41 @@ class RocksDB( } } + private def encodeStateRowWithPrefix( + data: Array[Byte], + cfName: String): Array[Byte] = { + // Create result array big enough for all prefixes plus data + val result = new Array[Byte](StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES + data.length) + val offset = Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES + + val cfInfo = getColumnFamilyInfo(cfName) + Platform.putShort(result, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + + // Write the actual data + Platform.copyMemory( + data, Platform.BYTE_ARRAY_OFFSET, + result, offset, + data.length + ) + + result + } + + private def decodeStateRowWithPrefix(data: Array[Byte]): (Array[Byte], String) = { + val cfId = Platform.getShort(data, Platform.BYTE_ARRAY_OFFSET) + val cfName = colFamilyIdToNameMap.get(cfId) + val offset = Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES + + val key = new Array[Byte](data.length - StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.copyMemory( + data, offset, + key, Platform.BYTE_ARRAY_OFFSET, + key.length + ) + + (key, cfName) + } + /** * Get the value for the given key if present, or null. * @note This will return the last written value even if it was uncommitted. @@ -668,12 +703,12 @@ class RocksDB( def get( key: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = { + var keyWithPrefix = key if (useColumnFamilies) { - val cfInfo = getColumnFamilyInfo(cfName) - Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + keyWithPrefix = encodeStateRowWithPrefix(key, cfName) } - db.get(readOptions, key) + db.get(readOptions, keyWithPrefix) } /** @@ -684,13 +719,13 @@ class RocksDB( key: Array[Byte], value: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + var keyWithPrefix = key if (useColumnFamilies) { - val cfInfo = getColumnFamilyInfo(cfName) - Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) - + keyWithPrefix = encodeStateRowWithPrefix(key, cfName) if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, key) + val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { + val cfInfo = getColumnFamilyInfo(cfName) if (cfInfo.isInternal) { numInternalKeysOnWritingVersion += 1 } else { @@ -700,15 +735,15 @@ class RocksDB( } } else { if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, key) + val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { numKeysOnWritingVersion += 1 } } } - db.put(writeOptions, key, value) - changelogWriter.foreach(_.put(key, value)) + db.put(writeOptions, keyWithPrefix, value) + changelogWriter.foreach(_.put(keyWithPrefix, value)) } /** @@ -726,13 +761,14 @@ class RocksDB( key: Array[Byte], value: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + var keyWithPrefix = key if (useColumnFamilies) { - val cfInfo = getColumnFamilyInfo(cfName) - Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + keyWithPrefix = encodeStateRowWithPrefix(key, cfName) if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, key) + val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { + val cfInfo = getColumnFamilyInfo(cfName) if (cfInfo.isInternal) { numInternalKeysOnWritingVersion += 1 } else { @@ -742,16 +778,15 @@ class RocksDB( } } else { if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, key) + val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { numKeysOnWritingVersion += 1 } } } - db.merge(writeOptions, key, value) - - changelogWriter.foreach(_.merge(key, value)) + db.merge(writeOptions, keyWithPrefix, value) + changelogWriter.foreach(_.merge(keyWithPrefix, value)) } /** @@ -759,13 +794,15 @@ class RocksDB( * @note This update is not committed to disk until commit() is called. */ def remove(key: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + var keyWithPrefix = key + if (useColumnFamilies) { - val cfInfo = getColumnFamilyInfo(cfName) - Platform.putShort(key, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) + keyWithPrefix = encodeStateRowWithPrefix(key, cfName) if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, key) + val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { + val cfInfo = getColumnFamilyInfo(cfName) if (cfInfo.isInternal) { numInternalKeysOnWritingVersion -= 1 } else { @@ -775,15 +812,15 @@ class RocksDB( } } else { if (conf.trackTotalNumberOfRows) { - val value = db.get(readOptions, key) + val value = db.get(readOptions, keyWithPrefix) if (value != null) { numKeysOnWritingVersion -= 1 } } } - db.delete(writeOptions, key) - changelogWriter.foreach(_.delete(key)) + db.delete(writeOptions, keyWithPrefix) + changelogWriter.foreach(_.delete(keyWithPrefix)) } /** @@ -803,7 +840,13 @@ class RocksDB( new NextIterator[ByteArrayPair] { override protected def getNext(): ByteArrayPair = { if (iter.isValid) { - byteArrayPair.set(iter.key, iter.value) + + var key = iter.key + if (useColumnFamilies) { + key = decodeStateRowWithPrefix(iter.key)._1 + } + + byteArrayPair.set(key, iter.value) iter.next() byteArrayPair } else { @@ -835,8 +878,7 @@ class RocksDB( } } else { while (iter.isValid) { - val cfId = Platform.getShort(iter.key, Platform.BYTE_ARRAY_OFFSET) - val cfName = colFamilyIdToNameMap.get(cfId) + val (_, cfName) = decodeStateRowWithPrefix(iter.key) val cfInfo = getColumnFamilyInfo(cfName) if (cfInfo.isInternal) { internalKeys += 1 @@ -853,9 +895,16 @@ class RocksDB( } } - def prefixScan(prefix: Array[Byte]): Iterator[ByteArrayPair] = { + def prefixScan( + prefix: Array[Byte], + cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = { val iter = db.newIterator() - iter.seek(prefix) + var updatedPrefix = prefix + + if (useColumnFamilies) { + updatedPrefix = encodeStateRowWithPrefix(prefix, cfName) + } + iter.seek(updatedPrefix) // Attempt to close this iterator if there is a task failure, or a task interruption. Option(TaskContext.get()).foreach { tc => @@ -864,8 +913,13 @@ class RocksDB( new NextIterator[ByteArrayPair] { override protected def getNext(): ByteArrayPair = { - if (iter.isValid && iter.key().take(prefix.length).sameElements(prefix)) { - byteArrayPair.set(iter.key, iter.value) + if (iter.isValid && iter.key().take(updatedPrefix.length).sameElements(updatedPrefix)) { + var key = iter.key + if (useColumnFamilies) { + key = decodeStateRowWithPrefix(iter.key)._1 + } + + byteArrayPair.set(key, iter.value) iter.next() byteArrayPair } else { @@ -879,14 +933,6 @@ class RocksDB( } } - def prefixScan(cfName: String): Iterator[ByteArrayPair] = { - assert(useColumnFamilies == true) - val cfBytes = new Array[Byte](StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) - val cfInfo = getColumnFamilyInfo(cfName) - Platform.putShort(cfBytes, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) - prefixScan(cfBytes) - } - /** * Commit all the updates made as a version to DFS. The steps it needs to do to commits are: * - Flush all changes to disk diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index 652940d256d38..69fc5743a7f39 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -1011,61 +1011,6 @@ class AvroStateEncoder( } } -class StateRowPrefixEncoder(useColumnFamilies: Boolean) { - - private val numColFamilyBytes = if (useColumnFamilies) { - StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES - } else { - 0 - } - - def getNumPrefixBytes: Int = numColFamilyBytes - - val out = new ByteArrayOutputStream - - /** - * Encodes a state row by adding schema and column family ID prefixes if enabled. - * - * @param data The byte array containing the data to be prefixed - * @return A new byte array containing the prefixed data. If no prefixing is needed - * (neither schema evolution nor column families are enabled), returns a copy - * of the input array to maintain consistency with the prefixed case. - */ - def encodeStateRowWithPrefix(data: Array[Byte]): Array[Byte] = { - // Create result array big enough for all prefixes plus data - val result = new Array[Byte](getNumPrefixBytes + data.length) - var offset = Platform.BYTE_ARRAY_OFFSET - - // Write column family ID if enabled - if (useColumnFamilies) { - offset += StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES - } - - // Write the actual data - Platform.copyMemory( - data, Platform.BYTE_ARRAY_OFFSET, - result, offset, - data.length - ) - - result - } - - def decodeStateRowData(stateRow: Array[Byte]): Array[Byte] = { - val offset = Platform.BYTE_ARRAY_OFFSET + getNumPrefixBytes - - // Extract the actual data - val dataLength = stateRow.length - getNumPrefixBytes - val data = new Array[Byte](dataLength) - Platform.copyMemory( - stateRow, offset, - data, Platform.BYTE_ARRAY_OFFSET, - dataLength - ) - data - } -} - /** * Factory object for creating state encoders used by RocksDB state store. * @@ -1125,7 +1070,7 @@ class PrefixKeyScanStateEncoder( keySchema: StructType, numColsPrefixKey: Int, useColumnFamilies: Boolean = false) - extends StateRowPrefixEncoder(useColumnFamilies) with RocksDBKeyStateEncoder with Logging { + extends RocksDBKeyStateEncoder with Logging { private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { keySchema.zipWithIndex.take(numColsPrefixKey) @@ -1171,13 +1116,11 @@ class PrefixKeyScanStateEncoder( remainingEncoded.length ) - // Add state row prefix using encoder - encodeStateRowWithPrefix(combinedData) + combinedData } override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { - // First decode the metadata prefixes and get the actual key data - val keyData = decodeStateRowData(keyBytes) + val keyData = keyBytes // Get prefix key length from the start of the actual key data val prefixKeyEncodedLen = Platform.getInt(keyData, Platform.BYTE_ARRAY_OFFSET) @@ -1219,8 +1162,7 @@ class PrefixKeyScanStateEncoder( dataWithLength, Platform.BYTE_ARRAY_OFFSET + 4, prefixKeyEncoded.length ) - - encodeStateRowWithPrefix(dataWithLength) + dataWithLength } override def supportPrefixKeyScan: Boolean = true @@ -1263,7 +1205,7 @@ class RangeKeyScanStateEncoder( keySchema: StructType, orderingOrdinals: Seq[Int], useColumnFamilies: Boolean = false) - extends StateRowPrefixEncoder(useColumnFamilies) with RocksDBKeyStateEncoder with Logging { + extends RocksDBKeyStateEncoder with Logging { private val rangeScanKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { orderingOrdinals.map { ordinal => @@ -1364,12 +1306,11 @@ class RangeKeyScanStateEncoder( remainingEncoded.length ) - encodeStateRowWithPrefix(combinedData) + combinedData } override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { - // First decode metadata prefixes to get the actual key data - val keyData = decodeStateRowData(keyBytes) + val keyData = keyBytes // Get range scan key length and extract it val prefixKeyEncodedLen = Platform.getInt(keyData, Platform.BYTE_ARRAY_OFFSET) @@ -1418,7 +1359,7 @@ class RangeKeyScanStateEncoder( rangeScanKeyEncoded.length ) - encodeStateRowWithPrefix(dataWithLength) + dataWithLength } override def supportPrefixKeyScan: Boolean = true @@ -1440,11 +1381,11 @@ class NoPrefixKeyStateEncoder( dataEncoder: RocksDBDataEncoder, keySchema: StructType, useColumnFamilies: Boolean = false) - extends StateRowPrefixEncoder(useColumnFamilies) with RocksDBKeyStateEncoder with Logging { + extends RocksDBKeyStateEncoder with Logging { override def encodeKey(row: UnsafeRow): Array[Byte] = { if (!useColumnFamilies) { - encodeStateRowWithPrefix(dataEncoder.encodeKey(row)) + dataEncoder.encodeKey(row) } else { // First encode the row with the data encoder val rowBytes = dataEncoder.encodeKey(row) @@ -1458,18 +1399,17 @@ class NoPrefixKeyStateEncoder( rowBytes.length ) - encodeStateRowWithPrefix(dataWithVersion) + dataWithVersion } } override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { if (!useColumnFamilies) { - dataEncoder.decodeKey(decodeStateRowData(keyBytes)) + dataEncoder.decodeKey(keyBytes) } else if (keyBytes == null) { null } else { - // First decode the metadata prefixes - val dataWithVersion = decodeStateRowData(keyBytes) + val dataWithVersion = keyBytes // Skip version byte to get to actual data val dataLength = dataWithVersion.length - STATE_ENCODING_NUM_VERSION_BYTES diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index a637895e60f70..a8a61b582b05c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -175,7 +175,7 @@ private[sql] class RocksDBStateStoreProvider // we'll need to do prefixScan on the default column family with the same column // family id prefix to get all rows stored in a given virtual column family if (useColumnFamilies) { - rocksDB.prefixScan(colFamilyName).map { kv => + rocksDB.prefixScan(Array.empty[Byte], colFamilyName).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) if (!isValidated && rowPair.value != null && !useColumnFamilies) { @@ -209,7 +209,7 @@ private[sql] class RocksDBStateStoreProvider val rowPair = new UnsafeRowPair() val prefix = kvEncoder._1.encodePrefixKey(prefixKey) - rocksDB.prefixScan(prefix).map { kv => + rocksDB.prefixScan(prefix, colFamilyName).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) rowPair @@ -335,7 +335,7 @@ private[sql] class RocksDBStateStoreProvider val result = rocksDB.removeColFamilyIfExists(colFamilyName) if (result) { - rocksDB.prefixScan(colFamilyName).foreach { kv => + rocksDB.prefixScan(Array.empty[Byte], colFamilyName).foreach { kv => rocksDB.remove(kv.key) } keyValueEncoderMap.remove(colFamilyName) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index baa81281f8a72..1af507d6be6b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -554,6 +554,9 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 49) { db.load(version) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } db.put(version.toString, version.toString) db.commit() if ((version + 1) % 5 == 0) db.doMaintenance() @@ -786,6 +789,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 2) { db.load(version) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put(version.toString, version.toString) db.commit() } @@ -794,6 +802,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Roll back to version 1 and start to process data. for (version <- 1 to 3) { db.load(version) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put(version.toString, version.toString) db.commit() } @@ -806,6 +819,10 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => // Open the db to verify that the state in 4.zip is no corrupted. db.load(4) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } } } @@ -825,6 +842,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version - 1) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -843,10 +865,20 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } for (version <- 30 to 60) { db.load(version - 1) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -855,12 +887,22 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(changelogVersionsPresent(remoteDir) === (30 to 60)) for (version <- 1 to 60) { db.load(version, readOnly = true) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } // recommit 60 to ensure that acquireLock is released for maintenance for (version <- 60 to 60) { db.load(version - 1) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -878,6 +920,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Verify the content of retained versions. for (version <- 30 to 60) { db.load(version, readOnly = true) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } } @@ -1057,6 +1104,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession CreateAtomicTestManager.shouldFailInCreateAtomic = false for (version <- 1 to 10) { db.load(version - 1) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put(version.toString, version.toString) // update "1" -> "1", "2" -> "2", ... db.commit() } @@ -1065,6 +1117,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Fail commit for next version and verify that reloading resets the files CreateAtomicTestManager.shouldFailInCreateAtomic = true db.load(10) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put("11", "11") intercept[IOException] { quietly { @@ -1076,6 +1133,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Abort commit for next version and verify that reloading resets the files db.load(10) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put("11", "11") db.rollback() assert(db.load(10, readOnly = true).iterator().map(toStr).toSet === version10Data) @@ -1092,6 +1154,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => db.load(0) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put("foo", "bar") db.commit() // call close first and maintenance can be still be invoked in the context of the @@ -1111,6 +1178,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => db.load(0) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put("foo", "bar") db.commit() // maintenance can be invoked in the context of the maintenance task's thread pool @@ -1970,16 +2042,16 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // should always include sstFiles and numKeys checkJsonRoundtrip( RocksDBCheckpointMetadata(Seq.empty, 0L), - """{"sstFiles":[],"numKeys":0}""" + """{"sstFiles":[],"numKeys":0,"numInternalKeys":0}""" ) // shouldn't include the "logFiles" field in json when it's empty checkJsonRoundtrip( RocksDBCheckpointMetadata(sstFiles, 12345678901234L), - """{"sstFiles":[{"localFileName":"00001.sst","dfsSstFileName":"00001-uuid.sst","sizeBytes":12345678901234}],"numKeys":12345678901234}""" + """{"sstFiles":[{"localFileName":"00001.sst","dfsSstFileName":"00001-uuid.sst","sizeBytes":12345678901234}],"numKeys":12345678901234,"numInternalKeys":0}""" ) checkJsonRoundtrip( RocksDBCheckpointMetadata(sstFiles, logFiles, 12345678901234L), - """{"sstFiles":[{"localFileName":"00001.sst","dfsSstFileName":"00001-uuid.sst","sizeBytes":12345678901234}],"logFiles":[{"localFileName":"00001.log","dfsLogFileName":"00001-uuid.log","sizeBytes":12345678901234}],"numKeys":12345678901234}""") + """{"sstFiles":[{"localFileName":"00001.sst","dfsSstFileName":"00001-uuid.sst","sizeBytes":12345678901234}],"logFiles":[{"localFileName":"00001.log","dfsLogFileName":"00001-uuid.log","sizeBytes":12345678901234}],"numKeys":12345678901234,"numInternalKeys":0}""") // scalastyle:on line.size.limit } @@ -2189,6 +2261,10 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession val remoteDir = dir.getCanonicalPath withDB(remoteDir, conf = dbConf, useColumnFamilies = colFamiliesEnabled) { db => db.load(0) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.put("a", "1") db.commit() } @@ -3218,8 +3294,15 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession useColumnFamilies = useColumnFamilies) } db.load(version) + if (useColumnFamilies) { + logWarning(s"Creating default col family") + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } func(db) } finally { + if (useColumnFamilies && db != null) { + db.removeColFamilyIfExists(StateStore.DEFAULT_COL_FAMILY_NAME) + } if (db != null) { db.close() } From 2f026d03a82634048d2ac6d6095928bbf8d2aaa6 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 27 Dec 2024 16:21:49 -0800 Subject: [PATCH 07/37] Fix test --- .../execution/streaming/state/RocksDB.scala | 1 - .../streaming/state/RocksDBSuite.scala | 75 +++++++++++++++++-- 2 files changed, 67 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index b7632184c7b04..f53292f9ed67f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -211,7 +211,6 @@ class RocksDB( */ def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean): Short = { if (!checkColFamilyExists(colFamilyName)) { - logWarning(s"TEST: creating column family $colFamilyName") val newColumnFamilyId = maxColumnFamilyId.incrementAndGet().toShort colFamilyNameToInfoMap.putIfAbsent(colFamilyName, ColumnFamilyInfo(newColumnFamilyId, isInternal)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 1af507d6be6b8..850046c31ccf3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -732,30 +732,71 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { db.load(version) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() db.doMaintenance() } - // Snapshot should not be created because minDeltasForSnapshot = 3 - assert(snapshotVersionsPresent(remoteDir) === Seq.empty) + + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1)) + } else { + // Snapshot should not be created because minDeltasForSnapshot = 3 + assert(snapshotVersionsPresent(remoteDir) === Seq.empty) + } + assert(changelogVersionsPresent(remoteDir) == Seq(1, 2)) db.load(2) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } db.commit() db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3)) + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(3)) + } + db.load(3) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } for (version <- 3 to 7) { db.load(version) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() db.doMaintenance() } - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6)) + + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1, 4, 7)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6)) + } + for (version <- 8 to 17) { db.load(version) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() } db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) + + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1, 4, 7, 16)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) + } } // pick up from the last snapshot and the next upload will be for version 21 @@ -763,16 +804,35 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => db.load(18) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) + + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1, 4, 7, 16, 19)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18)) + } for (version <- 19 to 20) { db.load(version) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() } db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18, 21)) + + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1, 4, 7, 16, 19)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(3, 6, 18, 21)) + } } } @@ -3295,7 +3355,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } db.load(version) if (useColumnFamilies) { - logWarning(s"Creating default col family") db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) } func(db) From 09f0b6c461d349f31ec166b821b1c974f4de8cdf Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 27 Dec 2024 22:47:50 -0800 Subject: [PATCH 08/37] Fix test --- .../streaming/state/RocksDBSuite.scala | 70 +++++++++++++++++-- 1 file changed, 66 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 850046c31ccf3..25c68517d2262 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -581,7 +581,12 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession if (isChangelogCheckpointingEnabled) { assert(changelogVersionsPresent(remoteDir) === (1 to 50)) - assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === + Seq(1, 5, 10, 15, 20, 25, 30, 35, 40, 45, 50)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) + } } else { assert(changelogVersionsPresent(remoteDir) === Seq.empty) assert(snapshotVersionsPresent(remoteDir) === (1 to 50)) @@ -639,20 +644,46 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => db.load(0) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() for (version <- 1 to 2) { db.load(version) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() db.doMaintenance() } - assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) + + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1, 2, 3)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) + } assert(changelogVersionsPresent(remoteDir) == Seq(1, 2, 3)) for (version <- 3 to 4) { db.load(version) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() } - assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) + + if (colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(1, 2, 3)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(2, 3)) + } + assert(changelogVersionsPresent(remoteDir) == (1 to 5)) db.doMaintenance() // 3 is the latest snapshot <= maxSnapshotVersionPresent - minVersionsToRetain + 1 @@ -661,6 +692,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession for (version <- 5 to 7) { db.load(version) + + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } + db.commit() } assert(snapshotVersionsPresent(remoteDir) === Seq(3, 5)) @@ -1007,6 +1043,9 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version - 1) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -1024,18 +1063,38 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } + for (version <- 31 to 60) { db.load(version - 1) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() } assert(changelogVersionsPresent(remoteDir) === (1 to 30)) - assert(snapshotVersionsPresent(remoteDir) === (31 to 60)) + + var result: Seq[Long] = if (colFamiliesEnabled) { + Seq(1) + } else { + Seq.empty + } + + (31 to 60).foreach { i => + result = result :+ i + } + assert(snapshotVersionsPresent(remoteDir) === result) for (version <- 1 to 60) { db.load(version, readOnly = true) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } // Check that snapshots and changelogs get purged correctly. @@ -1045,6 +1104,9 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Verify the content of retained versions. for (version <- 41 to 60) { db.load(version, readOnly = true) + if (colFamiliesEnabled) { + db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } } From 4deab178f51b7d49d7872c22cdb885ad1a42267b Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 27 Dec 2024 22:53:30 -0800 Subject: [PATCH 09/37] Add comment --- .../sql/execution/streaming/state/RocksDBFileManager.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 8a20fde3b96b0..8020872f126a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -924,6 +924,11 @@ object RocksDBFileManagerMetrics { val EMPTY_METRICS = RocksDBFileManagerMetrics(0L, 0L, 0L, None) } +/** + * Case class to keep track of column family info within checkpoint metadata. + * @param cfId - virtual column family id + * @param isInternal - whether the column family is internal or not + */ case class ColumnFamilyInfo( cfId: Short, isInternal: Boolean) From 4b93a40cc2badfaad140bca19c76b918f7cef2cb Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 27 Dec 2024 23:16:59 -0800 Subject: [PATCH 10/37] Misc fix --- .../streaming/state/RocksDBStateStoreProvider.scala | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index a8a61b582b05c..c23324d2ceb5d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -93,7 +93,7 @@ private[sql] class RocksDBStateStoreProvider val kvEncoder = keyValueEncoderMap.get(colFamilyName) val value = - kvEncoder._2.decodeValue(rocksDB.get(kvEncoder._1.encodeKey(key))) + kvEncoder._2.decodeValue(rocksDB.get(kvEncoder._1.encodeKey(key), colFamilyName)) if (!isValidated && value != null && !useColumnFamilies) { StateStoreProvider.validateStateRowFormat( @@ -124,7 +124,7 @@ private[sql] class RocksDBStateStoreProvider verify(valueEncoder.supportsMultipleValuesPerKey, "valuesIterator requires a encoder " + "that supports multiple values for a single key.") - val encodedValues = rocksDB.get(keyEncoder.encodeKey(key)) + val encodedValues = rocksDB.get(keyEncoder.encodeKey(key), colFamilyName) valueEncoder.decodeValues(encodedValues) } @@ -141,7 +141,7 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - rocksDB.merge(keyEncoder.encodeKey(key), valueEncoder.encodeValue(value)) + rocksDB.merge(keyEncoder.encodeKey(key), valueEncoder.encodeValue(value), colFamilyName) } override def put(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = { @@ -151,7 +151,7 @@ private[sql] class RocksDBStateStoreProvider verifyColFamilyOperations("put", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - rocksDB.put(kvEncoder._1.encodeKey(key), kvEncoder._2.encodeValue(value)) + rocksDB.put(kvEncoder._1.encodeKey(key), kvEncoder._2.encodeValue(value), colFamilyName) } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { @@ -160,7 +160,7 @@ private[sql] class RocksDBStateStoreProvider verifyColFamilyOperations("remove", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - rocksDB.remove(kvEncoder._1.encodeKey(key)) + rocksDB.remove(kvEncoder._1.encodeKey(key), colFamilyName) } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { From 00271231c5c4de3004057c76012f5c1a9d844399 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Sun, 29 Dec 2024 20:47:42 -0800 Subject: [PATCH 11/37] Misc fix --- .../spark/sql/execution/streaming/state/RocksDB.scala | 6 ++++++ .../streaming/state/RocksDBStateStoreProvider.scala | 4 ---- 2 files changed, 6 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index f53292f9ed67f..edcd4b2bc9cf0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -229,6 +229,9 @@ class RocksDB( def removeColFamilyIfExists(colFamilyName: String): Boolean = { if (checkColFamilyExists(colFamilyName)) { shouldForceSnapshot.set(true) + prefixScan(Array.empty[Byte], colFamilyName).foreach { kv => + remove(kv.key, colFamilyName) + } val colFamilyInfo = colFamilyNameToInfoMap.get(colFamilyName) colFamilyNameToInfoMap.remove(colFamilyName) colFamilyIdToNameMap.remove(colFamilyInfo.cfId) @@ -493,6 +496,9 @@ class RocksDB( metadata.maxColumnFamilyId.foreach { maxId => maxColumnFamilyId.set(maxId) } + + createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + openDB() val (numKeys, numInternalKeys) = { if (!conf.trackTotalNumberOfRows) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index c23324d2ceb5d..e36a947d474cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -333,11 +333,7 @@ private[sql] class RocksDBStateStoreProvider verify(useColumnFamilies, "Column families are not supported in this store") val result = rocksDB.removeColFamilyIfExists(colFamilyName) - if (result) { - rocksDB.prefixScan(Array.empty[Byte], colFamilyName).foreach { kv => - rocksDB.remove(kv.key) - } keyValueEncoderMap.remove(colFamilyName) } result From c82b370fa70c27a59b5dcb83b4b064ddcaa454b7 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Sun, 29 Dec 2024 22:40:40 -0800 Subject: [PATCH 12/37] Fix --- .../apache/spark/sql/execution/streaming/state/RocksDB.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index edcd4b2bc9cf0..764c2b919ff17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -806,7 +806,7 @@ class RocksDB( if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) - if (oldValue == null) { + if (oldValue != null) { val cfInfo = getColumnFamilyInfo(cfName) if (cfInfo.isInternal) { numInternalKeysOnWritingVersion -= 1 From cb3248280ac9d0d9ab87608f361f6a76f803abb7 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Mon, 30 Dec 2024 12:13:10 -0800 Subject: [PATCH 13/37] Fix --- .../spark/sql/execution/streaming/state/RocksDB.scala | 6 ++++-- .../spark/sql/execution/streaming/state/RocksDBSuite.scala | 3 +-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 764c2b919ff17..d1233f6b25e98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -248,7 +248,7 @@ class RocksDB( * @return - true if the column family exists, false otherwise */ def checkColFamilyExists(colFamilyName: String): Boolean = { - colFamilyNameToInfoMap.containsKey(colFamilyName) + db != null && colFamilyNameToInfoMap.containsKey(colFamilyName) } // This method sets the internal column family metadata to @@ -497,7 +497,9 @@ class RocksDB( maxColumnFamilyId.set(maxId) } - createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + if (useColumnFamilies) { + createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) + } openDB() val (numKeys, numInternalKeys) = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 25c68517d2262..412d3cf4123b4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -582,8 +582,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession if (isChangelogCheckpointingEnabled) { assert(changelogVersionsPresent(remoteDir) === (1 to 50)) if (colFamiliesEnabled) { - assert(snapshotVersionsPresent(remoteDir) === - Seq(1, 5, 10, 15, 20, 25, 30, 35, 40, 45, 50)) + assert(snapshotVersionsPresent(remoteDir) === (1 to 50)) } else { assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) } From 1c427c16793da6003546eeade1dd097b3e06b9c4 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Mon, 30 Dec 2024 14:50:35 -0800 Subject: [PATCH 14/37] Fix CI --- .../state/RocksDBStateStoreProvider.scala | 59 +++++++++++++++---- 1 file changed, 46 insertions(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index e36a947d474cf..5e3cb103230d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.execution.streaming.{CheckpointFileManager, StreamExecution} import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.Platform import org.apache.spark.util.{NonFateSharingCache, Utils} private[sql] class RocksDBStateStoreProvider @@ -63,7 +64,7 @@ private[sql] class RocksDBStateStoreProvider useMultipleValuesPerKey: Boolean = false, isInternal: Boolean = false): Unit = { verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) - rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) + val cfId = rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) val dataEncoderCacheKey = StateRowEncoderCacheKey( queryRunId = getRunId(hadoopConf), operatorId = stateStoreId.operatorId, @@ -84,7 +85,7 @@ private[sql] class RocksDBStateStoreProvider valueSchema, useMultipleValuesPerKey ) - keyValueEncoderMap.putIfAbsent(colFamilyName, (keyEncoder, valueEncoder)) + keyValueEncoderMap.putIfAbsent(colFamilyName, (keyEncoder, valueEncoder, cfId)) } override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { @@ -379,10 +380,6 @@ private[sql] class RocksDBStateStoreProvider val dataEncoder = getDataEncoder( stateStoreEncoding, dataEncoderCacheKey, keyStateEncoderSpec, valueSchema) - if (useColumnFamilies) { - rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - val keyEncoder = RocksDBStateEncoder.getKeyEncoder( dataEncoder, keyStateEncoderSpec, @@ -392,7 +389,15 @@ private[sql] class RocksDBStateStoreProvider valueSchema, useMultipleValuesPerKey ) - keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, (keyEncoder, valueEncoder)) + + var cfId: Short = 0 + if (useColumnFamilies) { + cfId = rocksDB.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, + isInternal = false) + } + + keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, + (keyEncoder, valueEncoder, cfId)) } override def stateStoreId: StateStoreId = stateStoreId_ @@ -483,7 +488,7 @@ private[sql] class RocksDBStateStoreProvider } private val keyValueEncoderMap = new java.util.concurrent.ConcurrentHashMap[String, - (RocksDBKeyStateEncoder, RocksDBValueStateEncoder)] + (RocksDBKeyStateEncoder, RocksDBValueStateEncoder, Short)] private val multiColFamiliesDisabledStr = "multiple column families is disabled in " + "RocksDBStateStoreProvider" @@ -772,16 +777,43 @@ class RocksDBStateStoreChangeDataReader( endVersion: Long, compressionCodec: CompressionCodec, keyValueEncoderMap: - ConcurrentHashMap[String, (RocksDBKeyStateEncoder, RocksDBValueStateEncoder)], + ConcurrentHashMap[String, (RocksDBKeyStateEncoder, RocksDBValueStateEncoder, Short)], colFamilyNameOpt: Option[String] = None) extends StateStoreChangeDataReader( fm, stateLocation, startVersion, endVersion, compressionCodec, colFamilyNameOpt) { override protected var changelogSuffix: String = "changelog" + /** + * Encodes a virtual column family ID into a byte array suitable for RocksDB. + * + * This method creates a fixed-size byte array prefixed with the virtual column family ID, + * which is used to partition data within RocksDB. + * + * @param virtualColFamilyId The column family identifier to encode + * @return A byte array containing the encoded column family ID + */ + private def getColumnFamilyIdBytes(virtualColFamilyId: Short): Array[Byte] = { + val encodedBytes = new Array[Byte](StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId) + encodedBytes + } + + private def getExtractedKey(data: Array[Byte]): Array[Byte] = { + val offset = Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES + + val key = new Array[Byte](data.length - StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.copyMemory( + data, offset, + key, Platform.BYTE_ARRAY_OFFSET, + key.length + ) + key + } + override def getNext(): (RecordType.Value, UnsafeRow, UnsafeRow, Long) = { var currRecord: (RecordType.Value, Array[Byte], Array[Byte]) = null - val currEncoder: (RocksDBKeyStateEncoder, RocksDBValueStateEncoder) = + val currEncoder: (RocksDBKeyStateEncoder, RocksDBValueStateEncoder, Short) = keyValueEncoderMap.get(colFamilyNameOpt .getOrElse(StateStore.DEFAULT_COL_FAMILY_NAME)) @@ -797,15 +829,16 @@ class RocksDBStateStoreChangeDataReader( return null } - // TODO: fix this val nextRecord = reader.next() - val colFamilyIdBytes: Array[Byte] = new Array[Byte](2) + val colFamilyIdBytes: Array[Byte] = getColumnFamilyIdBytes(currEncoder._3) val endIndex = colFamilyIdBytes.size // Function checks for byte arrays being equal // from index 0 to endIndex - 1 (both inclusive) if (java.util.Arrays.equals(nextRecord._2, 0, endIndex, colFamilyIdBytes, 0, endIndex)) { - currRecord = nextRecord + val extractedKey = getExtractedKey(nextRecord._2) + val result = (nextRecord._1, extractedKey, nextRecord._3) + currRecord = result } } } else { From f9439a2a04926b202bc5e45f427e4369e06fd856 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Mon, 30 Dec 2024 18:11:08 -0800 Subject: [PATCH 15/37] Fix test --- .../spark/sql/streaming/TransformWithListStateTTLSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala index b188b92bdbb7c..073f59e6d955d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala @@ -236,7 +236,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest // The number of updated rows is the total across the last time assertNumStateRows // was called, and we only update numRowsUpdated for primary key updates. We ran 6 batches // and each wrote 3 primary keys, so the total number of updated rows is 6 * 3 = 18. - assertNumStateRows(total = 24, updated = 18) + assertNumStateRows(total = 10, updated = 18) ) } } @@ -552,7 +552,7 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest // // It's important to check with assertNumStateRows, since the InputEvents // only return values for the current grouping key, not the entirety of RocksDB. - assertNumStateRows(total = 4, updated = 4), + assertNumStateRows(total = 1, updated = 4), // The k1 calls should both return no values. However, the k2 calls should return // one record each. We put these into one AddData call since we want them all to From 0ee0796be7844131e1cb1b4c14e59524fdaa972b Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Mon, 30 Dec 2024 22:22:30 -0800 Subject: [PATCH 16/37] Update --- .../streaming/state/RocksDBSuite.scala | 140 ------------------ 1 file changed, 140 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 412d3cf4123b4..e14798d992c13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -554,9 +554,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 49) { db.load(version) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } db.put(version.toString, version.toString) db.commit() if ((version + 1) % 5 == 0) db.doMaintenance() @@ -643,19 +640,9 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => db.load(0) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() for (version <- 1 to 2) { db.load(version) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() db.doMaintenance() } @@ -669,11 +656,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession for (version <- 3 to 4) { db.load(version) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() } @@ -691,11 +673,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession for (version <- 5 to 7) { db.load(version) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() } assert(snapshotVersionsPresent(remoteDir) === Seq(3, 5)) @@ -767,11 +744,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 1) { db.load(version) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() db.doMaintenance() } @@ -785,9 +757,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(changelogVersionsPresent(remoteDir) == Seq(1, 2)) db.load(2) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } db.commit() db.doMaintenance() if (colFamiliesEnabled) { @@ -797,16 +766,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } db.load(3) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - for (version <- 3 to 7) { db.load(version) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() db.doMaintenance() } @@ -819,10 +780,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession for (version <- 8 to 17) { db.load(version) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() } db.doMaintenance() @@ -839,11 +796,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession enableStateStoreCheckpointIds = enableStateStoreCheckpointIds, versionToUniqueId = versionToUniqueId) { db => db.load(18) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() db.doMaintenance() @@ -855,10 +807,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession for (version <- 19 to 20) { db.load(version) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.commit() } db.doMaintenance() @@ -884,11 +832,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 0 to 2) { db.load(version) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put(version.toString, version.toString) db.commit() } @@ -897,11 +840,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Roll back to version 1 and start to process data. for (version <- 1 to 3) { db.load(version) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put(version.toString, version.toString) db.commit() } @@ -914,10 +852,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => // Open the db to verify that the state in 4.zip is no corrupted. db.load(4) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } } } @@ -937,11 +871,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version - 1) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -960,20 +889,10 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } for (version <- 30 to 60) { db.load(version - 1) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -982,22 +901,12 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(changelogVersionsPresent(remoteDir) === (30 to 60)) for (version <- 1 to 60) { db.load(version, readOnly = true) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } // recommit 60 to ensure that acquireLock is released for maintenance for (version <- 60 to 60) { db.load(version - 1) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -1015,11 +924,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Verify the content of retained versions. for (version <- 30 to 60) { db.load(version, readOnly = true) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } } @@ -1042,9 +946,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version - 1) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -1062,17 +963,11 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession versionToUniqueId = versionToUniqueId) { db => for (version <- 1 to 30) { db.load(version) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } for (version <- 31 to 60) { db.load(version - 1) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } db.put(version.toString, version.toString) db.remove((version - 1).toString) db.commit() @@ -1091,9 +986,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession assert(snapshotVersionsPresent(remoteDir) === result) for (version <- 1 to 60) { db.load(version, readOnly = true) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } // Check that snapshots and changelogs get purged correctly. @@ -1103,9 +995,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Verify the content of retained versions. for (version <- 41 to 60) { db.load(version, readOnly = true) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) } } @@ -1225,11 +1114,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession CreateAtomicTestManager.shouldFailInCreateAtomic = false for (version <- 1 to 10) { db.load(version - 1) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put(version.toString, version.toString) // update "1" -> "1", "2" -> "2", ... db.commit() } @@ -1238,11 +1122,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Fail commit for next version and verify that reloading resets the files CreateAtomicTestManager.shouldFailInCreateAtomic = true db.load(10) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put("11", "11") intercept[IOException] { quietly { @@ -1254,11 +1133,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Abort commit for next version and verify that reloading resets the files db.load(10) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put("11", "11") db.rollback() assert(db.load(10, readOnly = true).iterator().map(toStr).toSet === version10Data) @@ -1275,11 +1149,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => db.load(0) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put("foo", "bar") db.commit() // call close first and maintenance can be still be invoked in the context of the @@ -1299,11 +1168,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled, enableStateStoreCheckpointIds = enableStateStoreCheckpointIds) { db => db.load(0) - - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put("foo", "bar") db.commit() // maintenance can be invoked in the context of the maintenance task's thread pool @@ -2382,10 +2246,6 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession val remoteDir = dir.getCanonicalPath withDB(remoteDir, conf = dbConf, useColumnFamilies = colFamiliesEnabled) { db => db.load(0) - if (colFamiliesEnabled) { - db.createColFamilyIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, isInternal = false) - } - db.put("a", "1") db.commit() } From d5c6111041eda4e84397b3478708aa5e6f56cd6d Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 31 Dec 2024 22:56:50 -0800 Subject: [PATCH 17/37] Fix test --- .../streaming/state/RocksDBSuite.scala | 34 +++++++++++++++---- 1 file changed, 28 insertions(+), 6 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index e14798d992c13..4422f9822378e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -897,7 +897,13 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession db.remove((version - 1).toString) db.commit() } - assert(snapshotVersionsPresent(remoteDir) === (1 to 30)) + + if (enableStateStoreCheckpointIds && colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === (1 to 30) :+ 30 :+ 31) + } else { + assert(snapshotVersionsPresent(remoteDir) === (1 to 30)) + } + assert(changelogVersionsPresent(remoteDir) === (30 to 60)) for (version <- 1 to 60) { db.load(version, readOnly = true) @@ -913,18 +919,34 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } // Check that snapshots and changelogs get purged correctly. db.doMaintenance() - assert(snapshotVersionsPresent(remoteDir) === Seq(30, 60)) + + if (enableStateStoreCheckpointIds && colFamiliesEnabled) { + assert(snapshotVersionsPresent(remoteDir) === Seq(31, 60, 60)) + } else { + assert(snapshotVersionsPresent(remoteDir) === Seq(30, 60)) + } if (enableStateStoreCheckpointIds) { // recommit version 60 creates another changelog file with different unique id - assert(changelogVersionsPresent(remoteDir) === (30 to 60) :+ 60) + if (colFamiliesEnabled) { + assert(changelogVersionsPresent(remoteDir) === (31 to 60) :+ 60) + } else { + assert(changelogVersionsPresent(remoteDir) === (30 to 60) :+ 60) + } } else { assert(changelogVersionsPresent(remoteDir) === (30 to 60)) } // Verify the content of retained versions. - for (version <- 30 to 60) { - db.load(version, readOnly = true) - assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) + if (enableStateStoreCheckpointIds && colFamiliesEnabled) { + for (version <- 31 to 60) { + db.load(version, readOnly = true) + assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) + } + } else { + for (version <- 30 to 60) { + db.load(version, readOnly = true) + assert(db.iterator().map(toStr).toSet === Set((version.toString, version.toString))) + } } } } From e2a2121dc4abc8a9a931a35dc05d23313d437ba0 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 2 Jan 2025 12:10:37 -0800 Subject: [PATCH 18/37] Fix test --- .../state/RocksDBStateStoreProvider.scala | 8 ++- .../state/RocksDBStateStoreSuite.scala | 60 +++++++++++++++++++ .../streaming/state/StateStoreSuite.scala | 22 ++++--- 3 files changed, 79 insertions(+), 11 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 5e3cb103230d7..68c2abbdd5d65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -297,6 +297,7 @@ private[sql] class RocksDBStateStoreProvider CUSTOM_METRIC_COMPACT_WRITTEN_BYTES -> nativeOpsMetrics("totalBytesWrittenByCompaction"), CUSTOM_METRIC_FLUSH_WRITTEN_BYTES -> nativeOpsMetrics("totalBytesWrittenByFlush"), CUSTOM_METRIC_PINNED_BLOCKS_MEM_USAGE -> rocksDBMetrics.pinnedBlocksMemUsage, + CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES_KEYS -> rocksDBMetrics.numInternalKeys, CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES -> internalColFamilyCnt(), CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES -> externalColFamilyCnt() ) ++ rocksDBMetrics.zipFileBytesUncompressed.map(bytes => @@ -744,6 +745,9 @@ object RocksDBStateStoreProvider { val CUSTOM_METRIC_PINNED_BLOCKS_MEM_USAGE = StateStoreCustomSizeMetric( "rocksdbPinnedBlocksMemoryUsage", "RocksDB: memory usage for pinned blocks") + val CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES_KEYS = StateStoreCustomSizeMetric( + "rocksdbNumInternalColFamiliesKeys", + "RocksDB: number of internal keys for internal column families") val CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES = StateStoreCustomSizeMetric( "rocksdbNumExternalColumnFamilies", "RocksDB: number of external column families") @@ -765,8 +769,8 @@ object RocksDBStateStoreProvider { CUSTOM_METRIC_BYTES_WRITTEN, CUSTOM_METRIC_ITERATOR_BYTES_READ, CUSTOM_METRIC_STALL_TIME, CUSTOM_METRIC_TOTAL_COMPACT_TIME, CUSTOM_METRIC_COMPACT_READ_BYTES, CUSTOM_METRIC_COMPACT_WRITTEN_BYTES, CUSTOM_METRIC_FLUSH_WRITTEN_BYTES, - CUSTOM_METRIC_PINNED_BLOCKS_MEM_USAGE, CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES, - CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES) + CUSTOM_METRIC_PINNED_BLOCKS_MEM_USAGE, CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES_KEYS, + CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES, CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES) } /** [[StateStoreChangeDataReader]] implementation for [[RocksDBStateStoreProvider]] */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala index d99ab7d163329..90ba6333b4ce8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala @@ -1193,6 +1193,66 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } } + testWithColumnFamiliesAndEncodingTypes(s"numInternalKeys metrics", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + tryWithProviderResource( + newStoreProvider(useColumnFamilies = colFamiliesEnabled)) { provider => + if (colFamiliesEnabled) { + val store = provider.getStore(0) + + // create non-internal col family and add data + val cfName = "testColFamily" + store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema)) + put(store, "a", 0, 1, cfName) + put(store, "b", 0, 2, cfName) + put(store, "c", 0, 3, cfName) + put(store, "d", 0, 4, cfName) + put(store, "e", 0, 5, cfName) + + // create internal col family and add data + val internalCfName = "$testIndex" + store.createColFamilyIfAbsent(internalCfName, keySchema, valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema), isInternal = true) + put(store, "a", 0, 1, internalCfName) + put(store, "m", 0, 2, internalCfName) + put(store, "n", 0, 3, internalCfName) + put(store, "b", 0, 4, internalCfName) + + assert(store.commit() === 1) + // Commit and verify that the metrics are correct for internal and non-internal col families + assert(store.metrics.numKeys === 5) + val metricPair = store + .metrics.customMetrics.find(_._1.name == "rocksdbNumInternalColFamiliesKeys") + assert(metricPair.isDefined && metricPair.get._2 === 4) + assert(rowPairsToDataSet(store.iterator(cfName)) === + Set(("a", 0) -> 1, ("b", 0) -> 2, ("c", 0) -> 3, ("d", 0) -> 4, ("e", 0) -> 5)) + assert(rowPairsToDataSet(store.iterator(internalCfName)) === + Set(("a", 0) -> 1, ("m", 0) -> 2, ("n", 0) -> 3, ("b", 0) -> 4)) + + // Reload the store and remove some keys + val reloadedProvider = newStoreProvider(store.id, colFamiliesEnabled) + val reloadedStore = reloadedProvider.getStore(1) + reloadedStore.createColFamilyIfAbsent(cfName, keySchema, valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema)) + reloadedStore.createColFamilyIfAbsent(internalCfName, keySchema, valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema), isInternal = true) + remove(reloadedStore, _._1 == "b", cfName) + remove(reloadedStore, _._1 == "m", internalCfName) + assert(reloadedStore.commit() === 2) + // Commit and verify that the metrics are correct for internal and non-internal col families + assert(reloadedStore.metrics.numKeys === 4) + val metricPairUpdated = reloadedStore + .metrics.customMetrics.find(_._1.name == "rocksdbNumInternalColFamiliesKeys") + assert(metricPairUpdated.isDefined && metricPairUpdated.get._2 === 3) + assert(rowPairsToDataSet(reloadedStore.iterator(cfName)) === + Set(("a", 0) -> 1, ("c", 0) -> 3, ("d", 0) -> 4, ("e", 0) -> 5)) + assert(rowPairsToDataSet(reloadedStore.iterator(internalCfName)) === + Set(("a", 0) -> 1, ("n", 0) -> 3, ("b", 0) -> 4)) + } + } + } + test(s"validate rocksdb removeColFamilyIfExists correctness") { Seq( NoPrefixKeyStateEncoderSpec(keySchema), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 47dd77f1bb9fd..3503f83408221 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -1871,22 +1871,26 @@ object StateStoreTestsHelper { iterator.map(rowPairToDataPair).toSet } - def remove(store: StateStore, condition: ((String, Int)) => Boolean): Unit = { - store.iterator().foreach { rowPair => - if (condition(keyRowToData(rowPair.key))) store.remove(rowPair.key) + def remove(store: StateStore, condition: ((String, Int)) => Boolean, + colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + store.iterator(colFamilyName).foreach { rowPair => + if (condition(keyRowToData(rowPair.key))) store.remove(rowPair.key, colFamilyName) } } - def put(store: StateStore, key1: String, key2: Int, value: Int): Unit = { - store.put(dataToKeyRow(key1, key2), dataToValueRow(value)) + def put(store: StateStore, key1: String, key2: Int, value: Int, + colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + store.put(dataToKeyRow(key1, key2), dataToValueRow(value), colFamilyName) } - def merge(store: StateStore, key1: String, key2: Int, value: Int): Unit = { - store.merge(dataToKeyRow(key1, key2), dataToValueRow(value)) + def merge(store: StateStore, key1: String, key2: Int, value: Int, + colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + store.merge(dataToKeyRow(key1, key2), dataToValueRow(value), colFamilyName) } - def get(store: ReadStateStore, key1: String, key2: Int): Option[Int] = { - Option(store.get(dataToKeyRow(key1, key2))).map(valueRowToData) + def get(store: ReadStateStore, key1: String, key2: Int, + colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Option[Int] = { + Option(store.get(dataToKeyRow(key1, key2), colFamilyName)).map(valueRowToData) } def newDir(): String = Utils.createTempDir().toString From 600389ed6f29c2096503584d52ae81c00bea287a Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 2 Jan 2025 12:29:57 -0800 Subject: [PATCH 19/37] Add comments --- .../sql/execution/streaming/state/RocksDB.scala | 12 ++++++++++++ .../streaming/TransformWithListStateTTLSuite.scala | 7 +++++-- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index d1233f6b25e98..a38603b115a22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -668,6 +668,12 @@ class RocksDB( } } + /** + * Function to encode state row with virtual col family id prefix + * @param data - passed byte array to be stored in state store + * @param cfName - name of column family + * @return - encoded byte array with virtual column family id prefix + */ private def encodeStateRowWithPrefix( data: Array[Byte], cfName: String): Array[Byte] = { @@ -688,6 +694,12 @@ class RocksDB( result } + /** + * Function to decode state row with virtual col family id prefix + * @param data - passed byte array retrieved from state store + * @return - pair of decoded byte array without virtual column family id prefix + * and name of column family + */ private def decodeStateRowWithPrefix(data: Array[Byte]): (Array[Byte], String) = { val cfId = Platform.getShort(data, Platform.BYTE_ARRAY_OFFSET) val cfName = colFamilyIdToNameMap.get(cfId) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala index 073f59e6d955d..d04573becf1ae 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithListStateTTLSuite.scala @@ -228,10 +228,13 @@ class TransformWithListStateTTLSuite extends TransformWithStateTTLTest // - List state: 1 record in the primary, TTL, min, and count indexes // - Value state: 1 record in the primary, and 1 record in the TTL index // - // So in total, that amounts to 2t + 4 + 2 = 2t + 6 records. + // So in total, that amounts to 2t + 4 + 2 = 2t + 6 records. This is for internal and + // non-internal column families. For non-internal column families, the total records are + // t + 2. // // In this test, we have 2 unique keys, and each key occurs 3 times. Thus, the total number - // of keys in state is 2 * (2t + 6) where t = 3, which is 24. + // of keys in state is 2 * (2t + 6) where t = 3, which is 24. And the total number of + // records in the primary indexes are 2 * (t + 2) = 10. // // The number of updated rows is the total across the last time assertNumStateRows // was called, and we only update numRowsUpdated for primary key updates. We ran 6 batches From ddf177022d8e6fab7b1a60e52d1371721fa9689a Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 2 Jan 2025 14:42:52 -0800 Subject: [PATCH 20/37] Address Eric's comments --- .../execution/streaming/state/RocksDB.scala | 94 ++++++++++++------- 1 file changed, 61 insertions(+), 33 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index a38603b115a22..518f2dedee716 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -206,15 +206,35 @@ class RocksDB( colFamilyNameToInfoMap.get(cfName) } + private def getColumnFamilyNameForId(cfId: Short): String = { + colFamilyIdToNameMap.get(cfId) + } + + private def addToColFamilyMaps(cfName: String, cfId: Short, isInternal: Boolean): Unit = { + colFamilyNameToInfoMap.put(cfName, ColumnFamilyInfo(cfId, isInternal)) + colFamilyIdToNameMap.put(cfId, cfName) + } + + private def removeFromColFamilyMaps(cfName: String): Unit = { + val colFamilyInfo = colFamilyNameToInfoMap.get(cfName) + if (colFamilyInfo != null) { + colFamilyNameToInfoMap.remove(cfName) + colFamilyIdToNameMap.remove(colFamilyInfo.cfId) + } + } + + private def clearColFamilyMaps(): Unit = { + colFamilyNameToInfoMap.clear() + colFamilyIdToNameMap.clear() + } + /** * Create RocksDB column family, if not created already */ def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean): Short = { if (!checkColFamilyExists(colFamilyName)) { val newColumnFamilyId = maxColumnFamilyId.incrementAndGet().toShort - colFamilyNameToInfoMap.putIfAbsent(colFamilyName, - ColumnFamilyInfo(newColumnFamilyId, isInternal)) - colFamilyIdToNameMap.putIfAbsent(newColumnFamilyId, colFamilyName) + addToColFamilyMaps(colFamilyName, newColumnFamilyId, isInternal) shouldForceSnapshot.set(true) newColumnFamilyId } else { @@ -232,9 +252,7 @@ class RocksDB( prefixScan(Array.empty[Byte], colFamilyName).foreach { kv => remove(kv.key, colFamilyName) } - val colFamilyInfo = colFamilyNameToInfoMap.get(colFamilyName) - colFamilyNameToInfoMap.remove(colFamilyName) - colFamilyIdToNameMap.remove(colFamilyInfo.cfId) + removeFromColFamilyMaps(colFamilyName) true } else { false @@ -254,8 +272,7 @@ class RocksDB( // This method sets the internal column family metadata to // the default values it should be set to on load private def setInitialCFInfo(): Unit = { - colFamilyNameToInfoMap.clear() - colFamilyIdToNameMap.clear() + clearColFamilyMaps() shouldForceSnapshot.set(false) maxColumnFamilyId.set(0) } @@ -488,8 +505,7 @@ class RocksDB( setInitialCFInfo() metadata.columnFamilyMapping.foreach { mapping => mapping.foreach { case (colFamilyName, colFamilyInfo) => - colFamilyNameToInfoMap.putIfAbsent(colFamilyName, colFamilyInfo) - colFamilyIdToNameMap.putIfAbsent(colFamilyInfo.cfId, colFamilyName) + addToColFamilyMaps(colFamilyName, colFamilyInfo.cfId, colFamilyInfo.isInternal) } } @@ -702,7 +718,7 @@ class RocksDB( */ private def decodeStateRowWithPrefix(data: Array[Byte]): (Array[Byte], String) = { val cfId = Platform.getShort(data, Platform.BYTE_ARRAY_OFFSET) - val cfName = colFamilyIdToNameMap.get(cfId) + val cfName = getColumnFamilyNameForId(cfId) val offset = Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES val key = new Array[Byte](data.length - StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) @@ -722,9 +738,10 @@ class RocksDB( def get( key: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = { - var keyWithPrefix = key - if (useColumnFamilies) { - keyWithPrefix = encodeStateRowWithPrefix(key, cfName) + val keyWithPrefix = if (useColumnFamilies) { + encodeStateRowWithPrefix(key, cfName) + } else { + key } db.get(readOptions, keyWithPrefix) @@ -738,9 +755,13 @@ class RocksDB( key: Array[Byte], value: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - var keyWithPrefix = key + val keyWithPrefix = if (useColumnFamilies) { + encodeStateRowWithPrefix(key, cfName) + } else { + key + } + if (useColumnFamilies) { - keyWithPrefix = encodeStateRowWithPrefix(key, cfName) if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { @@ -780,10 +801,13 @@ class RocksDB( key: Array[Byte], value: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - var keyWithPrefix = key - if (useColumnFamilies) { - keyWithPrefix = encodeStateRowWithPrefix(key, cfName) + val keyWithPrefix = if (useColumnFamilies) { + encodeStateRowWithPrefix(key, cfName) + } else { + key + } + if (useColumnFamilies) { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { @@ -813,11 +837,13 @@ class RocksDB( * @note This update is not committed to disk until commit() is called. */ def remove(key: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - var keyWithPrefix = key + val keyWithPrefix = if (useColumnFamilies) { + encodeStateRowWithPrefix(key, cfName) + } else { + key + } if (useColumnFamilies) { - keyWithPrefix = encodeStateRowWithPrefix(key, cfName) - if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue != null) { @@ -859,10 +885,10 @@ class RocksDB( new NextIterator[ByteArrayPair] { override protected def getNext(): ByteArrayPair = { if (iter.isValid) { - - var key = iter.key - if (useColumnFamilies) { - key = decodeStateRowWithPrefix(iter.key)._1 + val key = if (useColumnFamilies) { + decodeStateRowWithPrefix(iter.key)._1 + } else { + iter.key } byteArrayPair.set(key, iter.value) @@ -918,11 +944,12 @@ class RocksDB( prefix: Array[Byte], cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = { val iter = db.newIterator() - var updatedPrefix = prefix - - if (useColumnFamilies) { - updatedPrefix = encodeStateRowWithPrefix(prefix, cfName) + val updatedPrefix = if (useColumnFamilies) { + encodeStateRowWithPrefix(prefix, cfName) + } else { + prefix } + iter.seek(updatedPrefix) // Attempt to close this iterator if there is a task failure, or a task interruption. @@ -933,9 +960,10 @@ class RocksDB( new NextIterator[ByteArrayPair] { override protected def getNext(): ByteArrayPair = { if (iter.isValid && iter.key().take(updatedPrefix.length).sameElements(updatedPrefix)) { - var key = iter.key - if (useColumnFamilies) { - key = decodeStateRowWithPrefix(iter.key)._1 + val key = if (useColumnFamilies) { + decodeStateRowWithPrefix(iter.key)._1 + } else { + iter.key } byteArrayPair.set(key, iter.value) From 7d21451545cc3da988b6834c3d6b2682f2bfe9d0 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 2 Jan 2025 15:08:17 -0800 Subject: [PATCH 21/37] Small fix --- .../apache/spark/sql/execution/streaming/state/RocksDB.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 518f2dedee716..37881d81a27f3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -211,8 +211,8 @@ class RocksDB( } private def addToColFamilyMaps(cfName: String, cfId: Short, isInternal: Boolean): Unit = { - colFamilyNameToInfoMap.put(cfName, ColumnFamilyInfo(cfId, isInternal)) - colFamilyIdToNameMap.put(cfId, cfName) + colFamilyNameToInfoMap.putIfAbsent(cfName, ColumnFamilyInfo(cfId, isInternal)) + colFamilyIdToNameMap.putIfAbsent(cfId, cfName) } private def removeFromColFamilyMaps(cfName: String): Unit = { From 8b7eadcfa17f8c78a7da16d903a7b063aca08344 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 2 Jan 2025 19:00:15 -0800 Subject: [PATCH 22/37] Fix test --- .../streaming/state/RocksDBStateStoreIntegrationSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala index f170de66ee9df..1f4fd7f795716 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreIntegrationSuite.scala @@ -107,8 +107,8 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest "rocksdbTotalBytesReadByCompaction", "rocksdbTotalBytesWrittenByCompaction", "rocksdbTotalCompactionLatencyMs", "rocksdbWriterStallLatencyMs", "rocksdbTotalBytesReadThroughIterator", "rocksdbTotalBytesWrittenByFlush", - "rocksdbPinnedBlocksMemoryUsage", "rocksdbNumExternalColumnFamilies", - "rocksdbNumInternalColumnFamilies")) + "rocksdbPinnedBlocksMemoryUsage", "rocksdbNumInternalColFamiliesKeys", + "rocksdbNumExternalColumnFamilies", "rocksdbNumInternalColumnFamilies")) } } finally { query.stop() From 8ffb78bc12d658654a4da784e985a341f3f034e0 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 21 Jan 2025 14:14:37 -0800 Subject: [PATCH 23/37] Fix test --- .../spark/sql/execution/streaming/state/RocksDBSuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index fb9c0d6040f86..e7a55519d706a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -382,7 +382,6 @@ class RocksDBStateEncoderSuite extends SparkFunSuite { valueSchema, Some(testProvider), Some(StateStore.DEFAULT_COL_FAMILY_NAME)) - new AvroStateEncoder(keyStateEncoderSpec, valueSchema, None, None) } private def createNoPrefixKeyEncoder(): RocksDBDataEncoder = { From 6f0eac063008d1ce9336dda2c297d162ac9e6bbd Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 21 Jan 2025 14:30:46 -0800 Subject: [PATCH 24/37] Fix test --- .../spark/sql/execution/streaming/state/RocksDBSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index e7a55519d706a..1aec8c75f844c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -628,7 +628,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession if (isChangelogCheckpointingEnabled) { assert(changelogVersionsPresent(remoteDir) === (1 to 50)) if (colFamiliesEnabled) { - assert(snapshotVersionsPresent(remoteDir) === (1 to 50)) + assert(snapshotVersionsPresent(remoteDir) === + Seq(1, 5, 10, 15, 20, 25, 30, 35, 40, 45, 50)) } else { assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) } From f6b5c0c6c40699565ac338948beba05cf727cf6c Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 21 Jan 2025 14:53:33 -0800 Subject: [PATCH 25/37] Address review comments --- .../spark/sql/execution/streaming/state/RocksDB.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 37881d81a27f3..78e212bafe237 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -229,7 +229,13 @@ class RocksDB( } /** - * Create RocksDB column family, if not created already + * Check if the column family exists with given name and create one if it doesn't. Users can + * create external column families storing user facing data as well as internal column families + * such as secondary indexes. Metrics for both of these types are tracked separately. + * + * @param colFamilyName - column family name + * @param isInternal - whether the column family is for internal use or not + * @return - virtual column family id */ def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean): Short = { if (!checkColFamilyExists(colFamilyName)) { From a0f599bee4210dd39f92c9b510fe4cd678b640e4 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 4 Feb 2025 11:15:32 -0800 Subject: [PATCH 26/37] Misc fix --- .../sql/execution/streaming/state/RocksDB.scala | 14 +------------- 1 file changed, 1 insertion(+), 13 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 78e212bafe237..31e851d0bb693 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -194,14 +194,6 @@ class RocksDB( @GuardedBy("acquireLock") private val shouldForceSnapshot: AtomicBoolean = new AtomicBoolean(false) - /** - * Check whether the column family name is for internal column families. - * - * @param cfName - column family name - * @return - true if the column family is for internal use, false otherwise - */ - private def checkInternalColumnFamilies(cfName: String): Boolean = cfName.charAt(0) == '_' - private def getColumnFamilyInfo(cfName: String): ColumnFamilyInfo = { colFamilyNameToInfoMap.get(cfName) } @@ -284,11 +276,7 @@ class RocksDB( } def getColFamilyCount(isInternal: Boolean): Long = { - if (isInternal) { - colFamilyNameToInfoMap.asScala.keys.toSeq.count(checkInternalColumnFamilies) - } else { - colFamilyNameToInfoMap.asScala.keys.toSeq.count(!checkInternalColumnFamilies(_)) - } + colFamilyNameToInfoMap.asScala.values.toSeq.count(_.isInternal == isInternal) } // Mapping of local SST files to DFS files for file reuse. From c0057b936cf3d3f1c727b1f6984ce9dfc922a5ed Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 6 Feb 2025 23:18:06 -0800 Subject: [PATCH 27/37] Fix comment --- .../spark/sql/execution/streaming/state/RocksDB.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 31e851d0bb693..e5d93a4a89dcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -916,10 +916,13 @@ class RocksDB( iter.next() } } else { + var currCfInfoOpt: Option[(String, ColumnFamilyInfo)] = None while (iter.isValid) { val (_, cfName) = decodeStateRowWithPrefix(iter.key) - val cfInfo = getColumnFamilyInfo(cfName) - if (cfInfo.isInternal) { + if (currCfInfoOpt.isEmpty || currCfInfoOpt.get._1 != cfName) { + currCfInfoOpt = Some((cfName, getColumnFamilyInfo(cfName))) + } + if (currCfInfoOpt.get._2.isInternal) { internalKeys += 1 } else { keys += 1 From fa95eeeedf5104265b1549c548691272a5fa7d11 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Fri, 7 Feb 2025 17:17:59 -0800 Subject: [PATCH 28/37] Fix issue --- .../streaming/state/RocksDBStateEncoder.scala | 12 +++---- .../state/RocksDBStateStoreProvider.scala | 4 ++- .../state/RocksDBStateStoreSuite.scala | 32 +++++++++---------- .../streaming/state/RocksDBSuite.scala | 8 ++--- 4 files changed, 29 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala index 6624c95efdcb8..c7b324ec32e62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala @@ -719,27 +719,27 @@ class UnsafeRowDataEncoder( * * @param keyStateEncoderSpec Specification for how to encode keys (prefix/range scan) * @param valueSchema Schema for the values to be encoded - * @param stateSchemaInfo Schema version information for both keys and values + * @param stateSchemaProvider Optional state schema provider + * @param columnFamilyName Column family name to be used */ class AvroStateEncoder( keyStateEncoderSpec: KeyStateEncoderSpec, valueSchema: StructType, stateSchemaProvider: Option[StateSchemaProvider], - columnFamilyName: Option[String] + columnFamilyName: String ) extends RocksDBDataEncoder(keyStateEncoderSpec, valueSchema) with Logging { - assert(columnFamilyName.isDefined) private val avroEncoder = createAvroEnc(keyStateEncoderSpec, valueSchema) // current schema IDs instantiated lazily // schema information private lazy val currentKeySchemaId: Short = getStateSchemaProvider.getCurrentStateSchemaId( - columnFamilyName.get, + columnFamilyName, isKey = true ) private lazy val currentValSchemaId: Short = getStateSchemaProvider.getCurrentStateSchemaId( - columnFamilyName.get, + columnFamilyName, isKey = false ) @@ -1261,7 +1261,7 @@ class AvroStateEncoder( val schemaIdRow = decodeStateSchemaIdRow(bytes) val writerSchema = getStateSchemaProvider.getSchemaMetadataValue( StateSchemaMetadataKey( - columnFamilyName.get, + columnFamilyName, schemaIdRow.schemaId, isKey = false ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index 17497ddabd7f5..df97a851b5e04 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -699,11 +699,13 @@ object RocksDBStateStoreProvider { new java.util.concurrent.Callable[RocksDBDataEncoder] { override def call(): RocksDBDataEncoder = { if (stateStoreEncoding == Avro.toString) { + assert(columnFamilyName.isDefined, + "Column family name must be defined for Avro encoding") new AvroStateEncoder( keyStateEncoderSpec, valueSchema, stateSchemaProvider, - columnFamilyName + columnFamilyName.get ) } else { new UnsafeRowDataEncoder( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala index 591a7e775424a..4d939db8796b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala @@ -646,7 +646,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Create test data @@ -669,7 +669,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Decode with evolved schema @@ -720,7 +720,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Create test data with null value @@ -747,7 +747,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Decode original value with evolved schema @@ -801,7 +801,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) val proj = UnsafeProjection.create(initialValueSchema) @@ -819,7 +819,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) val decoded = encoder2.decodeValue(encoded) @@ -866,7 +866,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) val proj = UnsafeProjection.create(initialValueSchema) @@ -884,7 +884,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) val decoded = encoder2.decodeValue(encoded) @@ -935,7 +935,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Create test data @@ -958,7 +958,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Decode with evolved schema @@ -1015,7 +1015,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Create and encode data with initial schema (IntegerType) @@ -1035,7 +1035,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Should successfully decode IntegerType as LongType @@ -1082,7 +1082,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Create and encode data with initial order @@ -1102,7 +1102,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Should decode with correct field values despite reordering @@ -1153,7 +1153,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), initialValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Create and encode data with initial schema (LongType) @@ -1173,7 +1173,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid NoPrefixKeyStateEncoderSpec(keySchema), evolvedValueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME) + StateStore.DEFAULT_COL_FAMILY_NAME ) // Attempting to decode Long as Int should fail diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 1aec8c75f844c..94ae48b42a221 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -381,7 +381,7 @@ class RocksDBStateEncoderSuite extends SparkFunSuite { keyStateEncoderSpec, valueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME)) + StateStore.DEFAULT_COL_FAMILY_NAME) } private def createNoPrefixKeyEncoder(): RocksDBDataEncoder = { @@ -426,7 +426,7 @@ class RocksDBStateEncoderSuite extends SparkFunSuite { withClue("Testing prefix scan encoding: ") { val prefixKeySpec = PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey = 2) val encoder = new AvroStateEncoder(prefixKeySpec, valueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME)) + StateStore.DEFAULT_COL_FAMILY_NAME) // Then encode just the remaining key portion (which should include schema ID) val remainingKeyRow = keyProj.apply(InternalRow(null, null, 3.14)) @@ -450,7 +450,7 @@ class RocksDBStateEncoderSuite extends SparkFunSuite { withClue("Testing range scan encoding: ") { val rangeScanSpec = RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals = Seq(0, 1)) val encoder = new AvroStateEncoder(rangeScanSpec, valueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME)) + StateStore.DEFAULT_COL_FAMILY_NAME) // Encode remaining key (non-ordering columns) // For range scan, the remaining key schema only contains columns NOT in orderingOrdinals @@ -558,7 +558,7 @@ class RocksDBStateEncoderSuite extends SparkFunSuite { val keySpec = NoPrefixKeyStateEncoderSpec(keySchema) val stateSchemaInfo = Some(StateSchemaInfo(keySchemaId = 0, valueSchemaId = 42)) val avroEncoder = new AvroStateEncoder(keySpec, valueSchema, Some(testProvider), - Some(StateStore.DEFAULT_COL_FAMILY_NAME)) + StateStore.DEFAULT_COL_FAMILY_NAME) val valueEncoder = new SingleValueStateEncoder(avroEncoder, valueSchema) // Encode value From 72bb09faa97308b50ec3120e4fa5b96a7e32e405 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 13 Feb 2025 22:46:36 -0800 Subject: [PATCH 29/37] Address comments --- .../execution/streaming/state/RocksDB.scala | 26 ++++++++++++-- .../streaming/state/RocksDBFileManager.scala | 36 ++++++++++++++++--- 2 files changed, 55 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index e5d93a4a89dcf..3d948a635120b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -492,14 +492,36 @@ class RocksDB( this } + /** + * Function to check if col family is internal or not based on information recorded in + * checkpoint metadata. + * @param cfName - column family name + * @param metadata - checkpoint metadata + * @return - type of column family (internal or otherwise) + */ + private def checkColFamilyType( + cfName: String, + metadata: RocksDBCheckpointMetadata): Boolean = { + if (metadata.columnFamilyTypeMap.isEmpty) { + false + } else { + metadata.columnFamilyTypeMap.get.get(cfName) match { + case Some(cfType) => + cfType + case None => + false + } + } + } + /** * Initialize key metrics based on the metadata loaded from DFS and open local RocksDB. */ private def openLocalRocksDB(metadata: RocksDBCheckpointMetadata): Unit = { setInitialCFInfo() metadata.columnFamilyMapping.foreach { mapping => - mapping.foreach { case (colFamilyName, colFamilyInfo) => - addToColFamilyMaps(colFamilyName, colFamilyInfo.cfId, colFamilyInfo.isInternal) + mapping.foreach { case (colFamilyName, cfId) => + addToColFamilyMaps(colFamilyName, cfId, checkColFamilyType(colFamilyName, metadata)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala index 9ece5686a29ff..bb1198dfccafc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBFileManager.scala @@ -258,8 +258,27 @@ class RocksDBFileManager( val (localImmutableFiles, localOtherFiles) = listRocksDBFiles(checkpointDir) val rocksDBFiles = saveImmutableFilesToDfs( version, localImmutableFiles, fileMapping, checkpointUniqueId) + + val colFamilyIdMapping: Option[Map[String, Short]] = if (columnFamilyMapping.isDefined) { + Some(columnFamilyMapping.get.map { + case (cfName, cfInfo) => + cfName -> cfInfo.cfId + }) + } else { + None + } + + val colFamilyTypeMapping: Option[Map[String, Boolean]] = if (columnFamilyMapping.isDefined) { + Some(columnFamilyMapping.get.map { + case (cfName, cfInfo) => + cfName -> cfInfo.isInternal + }) + } else { + None + } + val metadata = RocksDBCheckpointMetadata(rocksDBFiles, numKeys, numInternalKeys, - columnFamilyMapping, maxColumnFamilyId) + colFamilyIdMapping, colFamilyTypeMapping, maxColumnFamilyId) val metadataFile = localMetadataFile(checkpointDir) metadata.writeToFile(metadataFile) logInfo(log"Written metadata for version ${MDC(LogKeys.VERSION_NUM, version)}:\n" + @@ -942,7 +961,8 @@ case class RocksDBCheckpointMetadata( logFiles: Seq[RocksDBLogFile], numKeys: Long, numInternalKeys: Long, - columnFamilyMapping: Option[Map[String, ColumnFamilyInfo]] = None, + columnFamilyMapping: Option[Map[String, Short]] = None, + columnFamilyTypeMap: Option[Map[String, Boolean]] = None, maxColumnFamilyId: Option[Short] = None) { require(columnFamilyMapping.isDefined == maxColumnFamilyId.isDefined, @@ -1018,7 +1038,8 @@ object RocksDBCheckpointMetadata { rocksDBFiles: Seq[RocksDBImmutableFile], numKeys: Long, numInternalKeys: Long, - columnFamilyMapping: Option[Map[String, ColumnFamilyInfo]], + columnFamilyMapping: Option[Map[String, Short]], + columnFamilyTypeMap: Option[Map[String, Boolean]], maxColumnFamilyId: Option[Short]): RocksDBCheckpointMetadata = { val (sstFiles, logFiles) = rocksDBFiles.partition(_.isInstanceOf[RocksDBSstFile]) new RocksDBCheckpointMetadata( @@ -1027,6 +1048,7 @@ object RocksDBCheckpointMetadata { numKeys, numInternalKeys, columnFamilyMapping, + columnFamilyTypeMap, maxColumnFamilyId ) } @@ -1044,7 +1066,8 @@ object RocksDBCheckpointMetadata { rocksDBFiles: Seq[RocksDBImmutableFile], numKeys: Long, numInternalKeys: Long, - columnFamilyMapping: Map[String, ColumnFamilyInfo], + columnFamilyMapping: Map[String, Short], + columnFamilyTypeMap: Map[String, Boolean], maxColumnFamilyId: Short): RocksDBCheckpointMetadata = { val (sstFiles, logFiles) = rocksDBFiles.partition(_.isInstanceOf[RocksDBSstFile]) new RocksDBCheckpointMetadata( @@ -1053,6 +1076,7 @@ object RocksDBCheckpointMetadata { numKeys, numInternalKeys, Some(columnFamilyMapping), + Some(columnFamilyTypeMap), Some(maxColumnFamilyId) ) } @@ -1063,7 +1087,8 @@ object RocksDBCheckpointMetadata { logFiles: Seq[RocksDBLogFile], numKeys: Long, numInternalKeys: Long, - columnFamilyMapping: Map[String, ColumnFamilyInfo], + columnFamilyMapping: Map[String, Short], + columnFamilyTypeMap: Map[String, Boolean], maxColumnFamilyId: Short): RocksDBCheckpointMetadata = { new RocksDBCheckpointMetadata( sstFiles, @@ -1071,6 +1096,7 @@ object RocksDBCheckpointMetadata { numKeys, numInternalKeys, Some(columnFamilyMapping), + Some(columnFamilyTypeMap), Some(maxColumnFamilyId) ) } From b576cdcd432f67d6c20237221fbbe83d50a43fd4 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Thu, 13 Feb 2025 23:29:53 -0800 Subject: [PATCH 30/37] Add test --- .../execution/streaming/state/RocksDBSuite.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 94ae48b42a221..1fd446a5d2089 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -2212,6 +2212,21 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession checkJsonRoundtrip( RocksDBCheckpointMetadata(sstFiles, logFiles, 12345678901234L), """{"sstFiles":[{"localFileName":"00001.sst","dfsSstFileName":"00001-uuid.sst","sizeBytes":12345678901234}],"logFiles":[{"localFileName":"00001.log","dfsLogFileName":"00001-uuid.log","sizeBytes":12345678901234}],"numKeys":12345678901234,"numInternalKeys":0}""") + + // verify format without including column family type + val cfMapping: Option[scala.collection.Map[String, Short]] = Some(Map("cf1" -> 1, "cf2" -> 2)) + var cfTypeMap: Option[scala.collection.Map[String, Boolean]] = None + val maxCfId: Option[Short] = Some(2) + checkJsonRoundtrip( + RocksDBCheckpointMetadata(Seq.empty, 5L, 0L, cfMapping, cfTypeMap, maxCfId), + """{"sstFiles":[],"numKeys":5,"numInternalKeys":0,"columnFamilyMapping":{"cf1":1,"cf2":2},"maxColumnFamilyId":2}""") + + // verify format including column family type and non-zero internal keys + cfTypeMap = Some(Map("cf1" -> true, "cf2" -> false)) + checkJsonRoundtrip( + RocksDBCheckpointMetadata(Seq.empty, 3L, 2L, cfMapping, cfTypeMap, maxCfId), + """{"sstFiles":[],"numKeys":3,"numInternalKeys":2,"columnFamilyMapping":{"cf1":1,"cf2":2},"columnFamilyTypeMap":{"cf1":true,"cf2":false},"maxColumnFamilyId":2}""") + // scalastyle:on line.size.limit } From cea9c747eebffd160c758ea612439950d069d803 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Mon, 17 Feb 2025 23:57:12 -0800 Subject: [PATCH 31/37] Address comments --- .../sql/execution/streaming/state/RocksDB.scala | 17 ++++++++++++++--- .../state/RocksDBStateStoreProvider.scala | 5 +---- 2 files changed, 15 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 3d948a635120b..45ecd36469826 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -247,7 +247,7 @@ class RocksDB( def removeColFamilyIfExists(colFamilyName: String): Boolean = { if (checkColFamilyExists(colFamilyName)) { shouldForceSnapshot.set(true) - prefixScan(Array.empty[Byte], colFamilyName).foreach { kv => + iterator(colFamilyName).foreach { kv => remove(kv.key, colFamilyName) } removeFromColFamilyMaps(colFamilyName) @@ -499,7 +499,7 @@ class RocksDB( * @param metadata - checkpoint metadata * @return - type of column family (internal or otherwise) */ - private def checkColFamilyType( + private def isInternalColFamily( cfName: String, metadata: RocksDBCheckpointMetadata): Boolean = { if (metadata.columnFamilyTypeMap.isEmpty) { @@ -521,7 +521,7 @@ class RocksDB( setInitialCFInfo() metadata.columnFamilyMapping.foreach { mapping => mapping.foreach { case (colFamilyName, cfId) => - addToColFamilyMaps(colFamilyName, cfId, checkColFamilyType(colFamilyName, metadata)) + addToColFamilyMaps(colFamilyName, cfId, isInternalColFamily(colFamilyName, metadata)) } } @@ -920,6 +920,17 @@ class RocksDB( } } + /** + * Get an iterator of all committed and uncommitted key-value pairs for the given column family. + */ + def iterator(cfName: String): Iterator[ByteArrayPair] = { + if (!useColumnFamilies) { + iterator() + } else { + prefixScan(Array.empty[Byte], cfName) + } + } + private def countKeys(): (Long, Long) = { val iter = db.newIterator() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index df97a851b5e04..df1912c2f223d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -188,11 +188,8 @@ private[sql] class RocksDBStateStoreProvider val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() - // As Virtual Column Family attaches a column family prefix to the key row, - // we'll need to do prefixScan on the default column family with the same column - // family id prefix to get all rows stored in a given virtual column family if (useColumnFamilies) { - rocksDB.prefixScan(Array.empty[Byte], colFamilyName).map { kv => + rocksDB.iterator(colFamilyName).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) if (!isValidated && rowPair.value != null && !useColumnFamilies) { From 0880fdadd6796a07eb8ac425ab3f3b1f70d7567b Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 18 Feb 2025 13:13:17 -0800 Subject: [PATCH 32/37] Address comments --- .../execution/streaming/state/RocksDB.scala | 28 +---- .../state/RocksDBStateStoreProvider.scala | 105 +++++++++++++----- .../streaming/state/StateStore.scala | 2 - 3 files changed, 79 insertions(+), 56 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 45ecd36469826..c933a752f07e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -41,7 +41,6 @@ import org.apache.spark.TaskContext import org.apache.spark.internal.{LogEntry, Logging, LogKeys, MDC} import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.unsafe.Platform import org.apache.spark.util.{NextIterator, Utils} // RocksDB operations that could acquire/release the instance lock @@ -709,21 +708,8 @@ class RocksDB( private def encodeStateRowWithPrefix( data: Array[Byte], cfName: String): Array[Byte] = { - // Create result array big enough for all prefixes plus data - val result = new Array[Byte](StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES + data.length) - val offset = Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES - val cfInfo = getColumnFamilyInfo(cfName) - Platform.putShort(result, Platform.BYTE_ARRAY_OFFSET, cfInfo.cfId) - - // Write the actual data - Platform.copyMemory( - data, Platform.BYTE_ARRAY_OFFSET, - result, offset, - data.length - ) - - result + RocksDBStateStoreProvider.encodeStateRowWithPrefix(data, cfInfo.cfId) } /** @@ -733,17 +719,9 @@ class RocksDB( * and name of column family */ private def decodeStateRowWithPrefix(data: Array[Byte]): (Array[Byte], String) = { - val cfId = Platform.getShort(data, Platform.BYTE_ARRAY_OFFSET) + val cfId = RocksDBStateStoreProvider.getColumnFamilyBytesAsId(data) val cfName = getColumnFamilyNameForId(cfId) - val offset = Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES - - val key = new Array[Byte](data.length - StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.copyMemory( - data, offset, - key, Platform.BYTE_ARRAY_OFFSET, - key.length - ) - + val key = RocksDBStateStoreProvider.decodeStateRowWithPrefix(data) (key, cfName) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala index df1912c2f223d..cd9fdb9469d60 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala @@ -651,6 +651,7 @@ object RocksDBStateStoreProvider { // Version as a single byte that specifies the encoding of the row data in RocksDB val STATE_ENCODING_NUM_VERSION_BYTES = 1 val STATE_ENCODING_VERSION: Byte = 0 + val VIRTUAL_COL_FAMILY_PREFIX_BYTES = 2 val SCHEMA_ID_PREFIX_BYTES = 2 @@ -658,6 +659,78 @@ object RocksDBStateStoreProvider { private val AVRO_ENCODER_LIFETIME_HOURS = 1L private val DEFAULT_SCHEMA_IDS = StateSchemaInfo(0, 0) + /** + * Encodes a virtual column family ID into a byte array suitable for RocksDB. + * + * This method creates a fixed-size byte array prefixed with the virtual column family ID, + * which is used to partition data within RocksDB. + * + * @param virtualColFamilyId The column family identifier to encode + * @return A byte array containing the encoded column family ID + */ + def getColumnFamilyIdAsBytes(virtualColFamilyId: Short): Array[Byte] = { + val encodedBytes = new Array[Byte](RocksDBStateStoreProvider.VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId) + encodedBytes + } + + /** + * Function to encode state row with virtual col family id prefix + * @param data - passed byte array to be stored in state store + * @param vcfId - virtual column family id + * @return - encoded byte array with virtual column family id prefix + */ + def encodeStateRowWithPrefix( + data: Array[Byte], + vcfId: Short): Array[Byte] = { + // Create result array big enough for all prefixes plus data + val result = new Array[Byte](RocksDBStateStoreProvider.VIRTUAL_COL_FAMILY_PREFIX_BYTES + + data.length) + val offset = Platform.BYTE_ARRAY_OFFSET + + RocksDBStateStoreProvider.VIRTUAL_COL_FAMILY_PREFIX_BYTES + + Platform.putShort(result, Platform.BYTE_ARRAY_OFFSET, vcfId) + + // Write the actual data + Platform.copyMemory( + data, Platform.BYTE_ARRAY_OFFSET, + result, offset, + data.length + ) + + result + } + + /** + * Function to decode virtual column family id from byte array + * @param data - passed byte array retrieved from state store + * @return - virtual column family id + */ + def getColumnFamilyBytesAsId(data: Array[Byte]): Short = { + Platform.getShort(data, Platform.BYTE_ARRAY_OFFSET) + } + + /** + * Function to decode state row with virtual col family id prefix + * @param data - passed byte array retrieved from state store + * @return - pair of decoded byte array without virtual column family id prefix + * and name of column family + */ + def decodeStateRowWithPrefix(data: Array[Byte]): Array[Byte] = { + val offset = Platform.BYTE_ARRAY_OFFSET + + RocksDBStateStoreProvider.VIRTUAL_COL_FAMILY_PREFIX_BYTES + + val key = new Array[Byte](data.length - + RocksDBStateStoreProvider.VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.copyMemory( + data, offset, + key, Platform.BYTE_ARRAY_OFFSET, + key.length + ) + + key + } + // Add the cache at companion object level so it persists across provider instances private val dataEncoderCache: NonFateSharingCache[StateRowEncoderCacheKey, RocksDBDataEncoder] = NonFateSharingCache( @@ -832,33 +905,6 @@ class RocksDBStateStoreChangeDataReader( override protected var changelogSuffix: String = "changelog" - /** - * Encodes a virtual column family ID into a byte array suitable for RocksDB. - * - * This method creates a fixed-size byte array prefixed with the virtual column family ID, - * which is used to partition data within RocksDB. - * - * @param virtualColFamilyId The column family identifier to encode - * @return A byte array containing the encoded column family ID - */ - private def getColumnFamilyIdBytes(virtualColFamilyId: Short): Array[Byte] = { - val encodedBytes = new Array[Byte](StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId) - encodedBytes - } - - private def getExtractedKey(data: Array[Byte]): Array[Byte] = { - val offset = Platform.BYTE_ARRAY_OFFSET + StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES - - val key = new Array[Byte](data.length - StateStore.VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.copyMemory( - data, offset, - key, Platform.BYTE_ARRAY_OFFSET, - key.length - ) - key - } - override def getNext(): (RecordType.Value, UnsafeRow, UnsafeRow, Long) = { var currRecord: (RecordType.Value, Array[Byte], Array[Byte]) = null val currEncoder: (RocksDBKeyStateEncoder, RocksDBValueStateEncoder, Short) = @@ -878,13 +924,14 @@ class RocksDBStateStoreChangeDataReader( } val nextRecord = reader.next() - val colFamilyIdBytes: Array[Byte] = getColumnFamilyIdBytes(currEncoder._3) + val colFamilyIdBytes: Array[Byte] = + RocksDBStateStoreProvider.getColumnFamilyIdAsBytes(currEncoder._3) val endIndex = colFamilyIdBytes.size // Function checks for byte arrays being equal // from index 0 to endIndex - 1 (both inclusive) if (java.util.Arrays.equals(nextRecord._2, 0, endIndex, colFamilyIdBytes, 0, endIndex)) { - val extractedKey = getExtractedKey(nextRecord._2) + val extractedKey = RocksDBStateStoreProvider.decodeStateRowWithPrefix(nextRecord._2) val result = (nextRecord._1, extractedKey, nextRecord._3) currRecord = result } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index fa4bba4d1832f..8ba3fc37162c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -696,8 +696,6 @@ object StateStore extends Logging { val DEFAULT_COL_FAMILY_NAME = "default" - val VIRTUAL_COL_FAMILY_PREFIX_BYTES = 2 - @GuardedBy("loadedProviders") private val loadedProviders = new mutable.HashMap[StateStoreProviderId, StateStoreProvider]() From e51d37d26ca2d0927eaab6b495b7d1022573b8e2 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 18 Feb 2025 13:36:30 -0800 Subject: [PATCH 33/37] Misc fix --- .../execution/streaming/state/RocksDB.scala | 87 +++++++------------ 1 file changed, 30 insertions(+), 57 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index c933a752f07e3..484b5c033e07c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -742,28 +742,25 @@ class RocksDB( } /** - * Put the given value for the given key. - * @note This update is not committed to disk until commit() is called. + * Function to keep track of metrics updates around the number of keys in the store. + * @param keyWithPrefix - key with prefix + * @param cfName - column family name + * @param isPutOrMerge - flag to indicate if the operation is put or merge */ - def put( - key: Array[Byte], - value: Array[Byte], - cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - val keyWithPrefix = if (useColumnFamilies) { - encodeStateRowWithPrefix(key, cfName) - } else { - key - } - + private def handleMetricsUpdate( + keyWithPrefix: Array[Byte], + cfName: String, + isPutOrMerge: Boolean): Unit = { + val updateCount = if (isPutOrMerge) 1L else -1L if (useColumnFamilies) { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { val cfInfo = getColumnFamilyInfo(cfName) if (cfInfo.isInternal) { - numInternalKeysOnWritingVersion += 1 + numInternalKeysOnWritingVersion += updateCount } else { - numKeysOnWritingVersion += 1 + numKeysOnWritingVersion += updateCount } } } @@ -771,11 +768,27 @@ class RocksDB( if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) if (oldValue == null) { - numKeysOnWritingVersion += 1 + numKeysOnWritingVersion += updateCount } } } + } + /** + * Put the given value for the given key. + * @note This update is not committed to disk until commit() is called. + */ + def put( + key: Array[Byte], + value: Array[Byte], + cfName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { + val keyWithPrefix = if (useColumnFamilies) { + encodeStateRowWithPrefix(key, cfName) + } else { + key + } + + handleMetricsUpdate(keyWithPrefix, cfName, isPutOrMerge = true) db.put(writeOptions, keyWithPrefix, value) changelogWriter.foreach(_.put(keyWithPrefix, value)) } @@ -801,27 +814,7 @@ class RocksDB( key } - if (useColumnFamilies) { - if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, keyWithPrefix) - if (oldValue == null) { - val cfInfo = getColumnFamilyInfo(cfName) - if (cfInfo.isInternal) { - numInternalKeysOnWritingVersion += 1 - } else { - numKeysOnWritingVersion += 1 - } - } - } - } else { - if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, keyWithPrefix) - if (oldValue == null) { - numKeysOnWritingVersion += 1 - } - } - } - + handleMetricsUpdate(keyWithPrefix, cfName, isPutOrMerge = true) db.merge(writeOptions, keyWithPrefix, value) changelogWriter.foreach(_.merge(keyWithPrefix, value)) } @@ -837,27 +830,7 @@ class RocksDB( key } - if (useColumnFamilies) { - if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(readOptions, keyWithPrefix) - if (oldValue != null) { - val cfInfo = getColumnFamilyInfo(cfName) - if (cfInfo.isInternal) { - numInternalKeysOnWritingVersion -= 1 - } else { - numKeysOnWritingVersion -= 1 - } - } - } - } else { - if (conf.trackTotalNumberOfRows) { - val value = db.get(readOptions, keyWithPrefix) - if (value != null) { - numKeysOnWritingVersion -= 1 - } - } - } - + handleMetricsUpdate(keyWithPrefix, cfName, isPutOrMerge = false) db.delete(writeOptions, keyWithPrefix) changelogWriter.foreach(_.delete(keyWithPrefix)) } From 732e13ca891f723392479d4bc4a8240c0f24b275 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 18 Feb 2025 13:44:16 -0800 Subject: [PATCH 34/37] Fix issue --- .../spark/sql/execution/streaming/state/RocksDBSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 1fd446a5d2089..039f71fb8848b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -628,8 +628,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession if (isChangelogCheckpointingEnabled) { assert(changelogVersionsPresent(remoteDir) === (1 to 50)) if (colFamiliesEnabled) { - assert(snapshotVersionsPresent(remoteDir) === - Seq(1, 5, 10, 15, 20, 25, 30, 35, 40, 45, 50)) + assert(snapshotVersionsPresent(remoteDir) === Seq(1) ++ Range.inclusive(5, 10, 5)) } else { assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) } @@ -948,6 +947,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession } if (enableStateStoreCheckpointIds && colFamiliesEnabled) { + // This is because 30 is executed twice and snapshot does not overwrite in checkpoint v2 assert(snapshotVersionsPresent(remoteDir) === (1 to 30) :+ 30 :+ 31) } else { assert(snapshotVersionsPresent(remoteDir) === (1 to 30)) From e55a99df06d2c840585838d579b62b6718102c84 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 18 Feb 2025 13:51:42 -0800 Subject: [PATCH 35/37] Fix test --- .../spark/sql/execution/streaming/state/RocksDBSuite.scala | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 039f71fb8848b..2c7f2a72dbd03 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -1155,9 +1155,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession Seq.empty } - (31 to 60).foreach { i => - result = result :+ i - } + result ++ (31 to 60) assert(snapshotVersionsPresent(remoteDir) === result) for (version <- 1 to 60) { db.load(version, versionToUniqueId.get(version), readOnly = true) From 09d0ee00022243f2a0d550e72c520de75841dcbb Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 18 Feb 2025 20:54:45 -0800 Subject: [PATCH 36/37] Fix issue --- .../execution/streaming/state/RocksDB.scala | 20 +++++++++++++++++-- .../streaming/state/RocksDBSuite.scala | 2 ++ 2 files changed, 20 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala index 484b5c033e07c..bc7ff53d9af36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala @@ -741,6 +741,22 @@ class RocksDB( db.get(readOptions, keyWithPrefix) } + /** + * Function to check if value exists for a key or not depending on the operation type. + * @param oldValue - old value for the key + * @param isPutOrMerge - flag to indicate if the operation is put or merge + * @return - true if the value doesn't exist for putAndMerge operation and vice versa for remove + */ + private def checkExistingEntry( + oldValue: Array[Byte], + isPutOrMerge: Boolean): Boolean = { + if (isPutOrMerge) { + oldValue == null + } else { + oldValue != null + } + } + /** * Function to keep track of metrics updates around the number of keys in the store. * @param keyWithPrefix - key with prefix @@ -755,7 +771,7 @@ class RocksDB( if (useColumnFamilies) { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) - if (oldValue == null) { + if (checkExistingEntry(oldValue, isPutOrMerge)) { val cfInfo = getColumnFamilyInfo(cfName) if (cfInfo.isInternal) { numInternalKeysOnWritingVersion += updateCount @@ -767,7 +783,7 @@ class RocksDB( } else { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, keyWithPrefix) - if (oldValue == null) { + if (checkExistingEntry(oldValue, isPutOrMerge)) { numKeysOnWritingVersion += updateCount } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 2c7f2a72dbd03..78f0ef0be6ac4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -969,6 +969,8 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession // Check that snapshots and changelogs get purged correctly. db.doMaintenance() + // Behavior is slightly different when column families are enabled with checkpoint v2 + // since snapshot version 31 was created previously. if (enableStateStoreCheckpointIds && colFamiliesEnabled) { assert(snapshotVersionsPresent(remoteDir) === Seq(31, 60, 60)) } else { From 9088b362b92d854d8e702fdaa2013346f89fa296 Mon Sep 17 00:00:00 2001 From: Anish Shrigondekar Date: Tue, 18 Feb 2025 22:26:24 -0800 Subject: [PATCH 37/37] Fix test --- .../spark/sql/execution/streaming/state/RocksDBSuite.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala index 78f0ef0be6ac4..7d4614d599733 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBSuite.scala @@ -628,7 +628,7 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession if (isChangelogCheckpointingEnabled) { assert(changelogVersionsPresent(remoteDir) === (1 to 50)) if (colFamiliesEnabled) { - assert(snapshotVersionsPresent(remoteDir) === Seq(1) ++ Range.inclusive(5, 10, 5)) + assert(snapshotVersionsPresent(remoteDir) === Seq(1) ++ Range.inclusive(5, 50, 5)) } else { assert(snapshotVersionsPresent(remoteDir) === Range.inclusive(5, 50, 5)) } @@ -1157,7 +1157,9 @@ class RocksDBSuite extends AlsoTestWithRocksDBFeatures with SharedSparkSession Seq.empty } - result ++ (31 to 60) + (31 to 60).foreach { i => + result = result :+ i + } assert(snapshotVersionsPresent(remoteDir) === result) for (version <- 1 to 60) { db.load(version, versionToUniqueId.get(version), readOnly = true)