From 7179e33e127bbcf08df46df47738c6b3e5944b59 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 26 Jun 2024 14:14:12 -0700 Subject: [PATCH 01/23] a dirty working version --- .../apache/spark/sql/internal/SQLConf.scala | 9 + .../streaming/state/RocksDBStateEncoder.scala | 189 +++++++++++++++--- .../state/RocksDBStateStoreProvider.scala | 100 +++++++-- .../state/RocksDBStateStoreSuite.scala | 140 +++++++++++-- .../streaming/state/StateStoreSuite.scala | 39 ++-- 5 files changed, 399 insertions(+), 78 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 06e0c6eda5896..faee5a5890a05 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2168,6 +2168,15 @@ object SQLConf { .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2") .createWithDefault(2) + val STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED = + buildConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled") + .internal() + .doc("Whether structured streaming use virtual column family. Currently this is " + + "only supported with TransformWithState operator.") + .version("4.0.0") + .booleanConf + .createWithDefault(false) + val STREAMING_STOP_ACTIVE_RUN_ON_RESTART = buildConf("spark.sql.streaming.stopActiveRunOnRestart") .doc("Running multiple runs of the same streaming query concurrently is not supported. " + 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 80c228d15334d..92d9af8eac3f8 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 @@ -30,9 +30,18 @@ import org.apache.spark.unsafe.Platform sealed trait RocksDBKeyStateEncoder { def supportPrefixKeyScan: Boolean + def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] + + def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = + throw new IllegalArgumentException(s"Unsupported encodePrefixKey with colFamilyId arg") + def encodeKey(row: UnsafeRow): Array[Byte] - def decodeKey(keyBytes: Array[Byte]): UnsafeRow + + def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = + throw new IllegalArgumentException(s"Unsupported encodeKey with colFamilyId arg") + + def decodeKey(keyBytes: Array[Byte], hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow } sealed trait RocksDBValueStateEncoder { @@ -147,27 +156,54 @@ class PrefixKeyScanStateEncoder( // Reusable objects private val joinedRowOnKey = new JoinedRow() - override def encodeKey(row: UnsafeRow): Array[Byte] = { + private def encodeKeyWithColFamilyPrefix( + row: UnsafeRow, + hasVirtualColFamilyPrefix: Boolean, + colFamilyId: Long = -1L): Array[Byte] = { val prefixKeyEncoded = encodeUnsafeRow(extractPrefixKey(row)) val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) + val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + + val encodedBytes = new Array[Byte](prefixKeyEncoded.length + + remainingEncoded.length + 4 + offSetForColFamilyPrefix) + if (hasVirtualColFamilyPrefix) { + Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + } - val encodedBytes = new Array[Byte](prefixKeyEncoded.length + remainingEncoded.length + 4) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncoded.length) + Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + prefixKeyEncoded.length) Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4, prefixKeyEncoded.length) + encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + prefixKeyEncoded.length) // NOTE: We don't put the length of remainingEncoded as we can calculate later // on deserialization. Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + prefixKeyEncoded.length, + encodedBytes, + Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix + prefixKeyEncoded.length, remainingEncoded.length) encodedBytes } - override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { - val prefixKeyEncodedLen = Platform.getInt(keyBytes, Platform.BYTE_ARRAY_OFFSET) + override def encodeKey(row: UnsafeRow): Array[Byte] = { + encodeKeyWithColFamilyPrefix(row, false) + } + + override def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = { + encodeKeyWithColFamilyPrefix(row, true, colFamilyId) + } + + private def decodeKeyWithColFamilyPrefix( + keyBytes: Array[Byte], + hasVirtualColFamilyPrefix: Boolean): UnsafeRow = { + val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + + val prefixKeyEncodedLen = Platform.getInt( + keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) - Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4, prefixKeyEncoded, + Platform.copyMemory(keyBytes, + Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes @@ -175,7 +211,8 @@ class PrefixKeyScanStateEncoder( val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + - prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, + offSetForColFamilyPrefix + prefixKeyEncodedLen, + remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) val prefixKeyDecoded = decodeToUnsafeRow(prefixKeyEncoded, numFields = numColsPrefixKey) @@ -185,19 +222,44 @@ class PrefixKeyScanStateEncoder( restoreKeyProjection(joinedRowOnKey.withLeft(prefixKeyDecoded).withRight(remainingKeyDecoded)) } + override def decodeKey( + keyBytes: Array[Byte], + hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow = { + decodeKeyWithColFamilyPrefix(keyBytes, hasVirtualColFamilyPrefix) + } + private def extractPrefixKey(key: UnsafeRow): UnsafeRow = { prefixKeyProjection(key) } - override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { + private def encodePrefixKeyWithColFamilyPrefix( + prefixKey: UnsafeRow, + hasVirtualColFamilyPrefix: Boolean = false, + colFamilyId: Long = -1L): Array[Byte] = { + val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + val prefixKeyEncoded = encodeUnsafeRow(prefixKey) - val prefix = new Array[Byte](prefixKeyEncoded.length + 4) - Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncoded.length) + val prefix = new Array[Byte]( + prefixKeyEncoded.length + 4 + offSetForColFamilyPrefix) + if (hasVirtualColFamilyPrefix) { + Platform.putLong(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + } + Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + prefixKeyEncoded.length) Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefix, - Platform.BYTE_ARRAY_OFFSET + 4, prefixKeyEncoded.length) + Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + prefixKeyEncoded.length) prefix } + override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { + encodePrefixKeyWithColFamilyPrefix(prefixKey) + } + + override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = { + encodePrefixKeyWithColFamilyPrefix(prefixKey, true, colFamilyId) + } + override def supportPrefixKeyScan: Boolean = true } @@ -491,40 +553,73 @@ class RangeKeyScanStateEncoder( writer.getRow() } - override def encodeKey(row: UnsafeRow): Array[Byte] = { + private def encodeKeyWithColFamilyPrefix( + row: UnsafeRow, + hasVirtualColFamilyPrefix: Boolean = false, + colFamilyId: Long = -1L): Array[Byte] = { // This prefix key has the columns specified by orderingOrdinals val prefixKey = extractPrefixKey(row) val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) + val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + val result = if (orderingOrdinals.length < keySchema.length) { val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) - val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + remainingEncoded.length + 4) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET, rangeScanKeyEncoded.length) + val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + + remainingEncoded.length + 4 + offSetForColFamilyPrefix) + + if (hasVirtualColFamilyPrefix) { + Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + } + + Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4, rangeScanKeyEncoded.length) + encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + rangeScanKeyEncoded.length) // NOTE: We don't put the length of remainingEncoded as we can calculate later // on deserialization. Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + rangeScanKeyEncoded.length, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + + rangeScanKeyEncoded.length + offSetForColFamilyPrefix, remainingEncoded.length) encodedBytes } else { // if the num of ordering cols is same as num of key schema cols, we don't need to // encode the remaining key as it's empty. - val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + 4) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET, rangeScanKeyEncoded.length) + val encodedBytes = new Array[Byte]( + rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) + if (hasVirtualColFamilyPrefix) { + Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + } + + Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4, rangeScanKeyEncoded.length) + encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + rangeScanKeyEncoded.length) encodedBytes } result } - override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { + override def encodeKey(row: UnsafeRow): Array[Byte] = { + encodeKeyWithColFamilyPrefix(row, false) + } + + override def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = { + encodeKeyWithColFamilyPrefix(row, true, colFamilyId) + } + + private def decodeKeyWithColFamilyPrefix( + keyBytes: Array[Byte], + hasVirtualColFamilyPrefix: Boolean): UnsafeRow = { val prefixKeyEncodedLen = Platform.getInt(keyBytes, Platform.BYTE_ARRAY_OFFSET) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) - Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4, prefixKeyEncoded, - Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) + val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + + Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) val prefixKeyDecodedForRangeScan = decodeToUnsafeRow(prefixKeyEncoded, numFields = orderingOrdinals.length) @@ -532,10 +627,12 @@ class RangeKeyScanStateEncoder( if (orderingOrdinals.length < keySchema.length) { // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes - val remainingKeyEncodedLen = keyBytes.length - 4 - prefixKeyEncodedLen + val remainingKeyEncodedLen = keyBytes.length - 4 - + prefixKeyEncodedLen - offSetForColFamilyPrefix val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) - Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + + Platform.copyMemory(keyBytes, + Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix + prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) @@ -552,6 +649,12 @@ class RangeKeyScanStateEncoder( } } + override def decodeKey( + keyBytes: Array[Byte], + hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow = { + decodeKeyWithColFamilyPrefix(keyBytes, hasVirtualColFamilyPrefix) + } + override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4) @@ -586,13 +689,41 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) override def encodeKey(row: UnsafeRow): Array[Byte] = encodeUnsafeRow(row) + override def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = { + val bytesToEncode = row.getBytes + val encodedBytes = new Array[Byte](bytesToEncode.length + + STATE_ENCODING_NUM_VERSION_BYTES + 8) + + Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET, STATE_ENCODING_VERSION) + // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. + Platform.copyMemory( + bytesToEncode, Platform.BYTE_ARRAY_OFFSET, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES, + bytesToEncode.length) + encodedBytes + } + /** * Decode byte array for a key to a UnsafeRow. * @note The UnsafeRow returned is reused across calls, and the UnsafeRow just points to * the given byte array. */ - override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { - decodeToUnsafeRow(keyBytes, keyRow) + override def decodeKey( + keyBytes: Array[Byte], + hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow = { + if (hasVirtualColFamilyPrefix) { + if (keyBytes != null) { + // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. + keyRow.pointTo( + keyBytes, + Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + 8, + keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - 8) + keyRow + } else { + null + } + } else decodeToUnsafeRow(keyBytes, keyRow) } override def supportPrefixKeyScan: Boolean = 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 e7fc9f56dd9eb..80c1e79b3ff46 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 @@ -28,7 +28,9 @@ import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors +import org.apache.spark.sql.internal.SQLConf.STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED import org.apache.spark.sql.types.StructType +import org.apache.spark.unsafe.Platform import org.apache.spark.util.Utils private[sql] class RocksDBStateStoreProvider @@ -60,6 +62,15 @@ private[sql] class RocksDBStateStoreProvider s"Failed to create column family with reserved_name=$colFamilyName") verify(useColumnFamilies, "Column families are not supported in this store") rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) + + val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf) + useVirtualColumnFamily = sparkConf.get(STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED) + if (useVirtualColumnFamily) { + // if use virtual column family, then use default col family, no need to create new + rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) + colFamilyToLongMap.putIfAbsent(colFamilyName, scala.util.Random.nextLong()) + } + keyValueEncoderMap.putIfAbsent(colFamilyName, (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec), RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) @@ -68,8 +79,15 @@ private[sql] class RocksDBStateStoreProvider override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val value = kvEncoder._2.decodeValue( - rocksDB.get(kvEncoder._1.encodeKey(key), colFamilyName)) + val value = if (useVirtualColumnFamily) { + kvEncoder._2.decodeValue( + rocksDB.get(kvEncoder._1.encodeKey(key, + colFamilyToLongMap.get(colFamilyName)))) + } else { + kvEncoder._2.decodeValue( + rocksDB.get(kvEncoder._1.encodeKey(key), colFamilyName)) + } + if (!isValidated && value != null && !useColumnFamilies) { StateStoreProvider.validateStateRowFormat( key, keySchema, value, valueSchema, storeConf) @@ -119,29 +137,60 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot put a null value") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - rocksDB.put(kvEncoder._1.encodeKey(key), - kvEncoder._2.encodeValue(value), colFamilyName) + if (useVirtualColumnFamily) { + rocksDB.put(kvEncoder._1.encodeKey(key, colFamilyToLongMap.get(colFamilyName)), + kvEncoder._2.encodeValue(value)) + } else { + rocksDB.put(kvEncoder._1.encodeKey(key), + kvEncoder._2.encodeValue(value), colFamilyName) + } } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { verify(state == UPDATING, "Cannot remove after already committed or aborted") verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - rocksDB.remove(kvEncoder._1.encodeKey(key), colFamilyName) + if (useVirtualColumnFamily) { + rocksDB.remove(kvEncoder._1.encodeKey(key, colFamilyToLongMap.get(colFamilyName))) + } else { + rocksDB.remove(kvEncoder._1.encodeKey(key), colFamilyName) + } } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { + // How to avoid memcpy here + def getIdBytes(id: Long): Array[Byte] = { + // Long is fixed to be 8 bytes + val encodedBytes = new Array[Byte](8) + Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) + encodedBytes + } + val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() - rocksDB.iterator(colFamilyName).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key), - kvEncoder._2.decodeValue(kv.value)) - if (!isValidated && rowPair.value != null && !useColumnFamilies) { - StateStoreProvider.validateStateRowFormat( - rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) - isValidated = true + if (useVirtualColumnFamily) { + val cfId: Long = colFamilyToLongMap.get(colFamilyName) + rocksDB.prefixScan(getIdBytes(cfId)).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), + kvEncoder._2.decodeValue(kv.value)) + if (!isValidated && rowPair.value != null && !useColumnFamilies) { + StateStoreProvider.validateStateRowFormat( + rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) + isValidated = true + } + rowPair + } + } else { + rocksDB.iterator(colFamilyName).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key), + kvEncoder._2.decodeValue(kv.value)) + if (!isValidated && rowPair.value != null && !useColumnFamilies) { + StateStoreProvider.validateStateRowFormat( + rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) + isValidated = true + } + rowPair } - rowPair } } @@ -151,12 +200,23 @@ private[sql] class RocksDBStateStoreProvider require(kvEncoder._1.supportPrefixKeyScan, "Prefix scan requires setting prefix key!") - val prefix = kvEncoder._1.encodePrefixKey(prefixKey) val rowPair = new UnsafeRowPair() - rocksDB.prefixScan(prefix, colFamilyName).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key), - kvEncoder._2.decodeValue(kv.value)) - rowPair + if (useVirtualColumnFamily) { + val prefix = kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToLongMap.get(colFamilyName)) + + rocksDB.prefixScan(prefix).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), + kvEncoder._2.decodeValue(kv.value)) + rowPair + } + } else { + val prefix = kvEncoder._1.encodePrefixKey(prefixKey) + + rocksDB.prefixScan(prefix, colFamilyName).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key), + kvEncoder._2.decodeValue(kv.value)) + rowPair + } } } @@ -364,6 +424,10 @@ private[sql] class RocksDBStateStoreProvider private val keyValueEncoderMap = new java.util.concurrent.ConcurrentHashMap[String, (RocksDBKeyStateEncoder, RocksDBValueStateEncoder)] + private val colFamilyToLongMap = new java.util.concurrent.ConcurrentHashMap[String, Long] + + private var useVirtualColumnFamily: Boolean = false + private def verify(condition: => Boolean, msg: String): Unit = { if (!condition) { throw new IllegalStateException(msg) } } 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 f3eb8a392d040..2428a504acaee 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 @@ -16,29 +16,30 @@ */ package org.apache.spark.sql.execution.streaming.state +// scalastyle:off +//import java.util.UUID -import java.util.UUID - -import scala.collection.immutable +//import scala.collection.immutable import scala.util.Random import org.apache.hadoop.conf.Configuration import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkUnsupportedOperationException} +//import org.apache.spark.{SparkConf, SparkUnsupportedOperationException} import org.apache.spark.io.CompressionCodec -import org.apache.spark.sql.LocalSparkSession.withSparkSession -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} +//import org.apache.spark.sql.LocalSparkSession.withSparkSession +//import org.apache.spark.sql.SparkSession +//import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +//import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.tags.ExtendedSQLTest -import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark.util.Utils +//import org.apache.spark.unsafe.Platform +//import org.apache.spark.unsafe.types.UTF8String +//import org.apache.spark.util.Utils +// scalastyle:on @ExtendedSQLTest class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvider] @@ -57,7 +58,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } import StateStoreTestsHelper._ - +/* testWithColumnFamilies(s"version encoding", TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => import RocksDBStateStoreProvider._ @@ -916,6 +917,121 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid assert(get(store, "a", 0).isEmpty) } } + } */ + + test("test virtual col family (noPrefix) - " + + "put & get & remove, iterator, prefixScan, remove col family") { + withSQLConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled" -> "true") { + tryWithProviderResource(newStoreProvider(keySchema, + NoPrefixKeyStateEncoderSpec(keySchema), true)) { provider => + val store = provider.getStore(0) + + // use non-default col family if column families are enabled + val cfName = "testColFamily" + store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema)) + + var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, + -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) + // put & get, iterator + timerTimestamps.foreach { ts => + val keyRow = if (ts < 0) { + dataToKeyRow("a", ts.toInt) + } else dataToKeyRow(ts.toString, ts.toInt) + val valueRow = dataToValueRow(1) + store.put(keyRow, valueRow, cfName) + assert(valueRowToData(store.get(keyRow, cfName)) === 1) + } + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + // remove + store.remove(dataToKeyRow(1L.toString, 1.toInt), cfName) + timerTimestamps = timerTimestamps.filter(_ != 1L) + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + store.commit() + } + } + } + + test("test virtual col family (prefix) - " + + "put & get & remove, iterator, prefixScan, remove col family") { + withSQLConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled" -> "true") { + tryWithProviderResource(newStoreProvider(keySchema, + PrefixKeyScanStateEncoderSpec(keySchema, 1), true)) { provider => + val store = provider.getStore(0) + + // use non-default col family if column families are enabled + val cfName = "testColFamily" + store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, + PrefixKeyScanStateEncoderSpec(keySchema, 1)) + + var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, + -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) + // put & get, iterator + timerTimestamps.foreach { ts => + val keyRow = if (ts < 0) { + dataToKeyRow("a", ts.toInt) + } else dataToKeyRow(ts.toString, ts.toInt) + val valueRow = dataToValueRow(1) + store.put(keyRow, valueRow, cfName) + assert(valueRowToData(store.get(keyRow, cfName)) === 1) + } + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + // remove + store.remove(dataToKeyRow(1L.toString, 1.toInt), cfName) + timerTimestamps = timerTimestamps.filter(_ != 1L) + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + // prefix scan + val keyRow = dataToPrefixKeyRow("a") + assert(store.prefixScan(keyRow, cfName).toSeq.length + == timerTimestamps.filter(_ < 0).length) + + store.commit() + } + } + } + + test("test virtual col family (rangeScan) - " + + "put & get & remove, iterator, prefixScan, remove col family") { + withSQLConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled" -> "true") { + tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), true)) { provider => + val store = provider.getStore(0) + + // use non-default col family if column families are enabled + val cfName = "testColFamily" + store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) + + var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, + -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) + // put & get, iterator + timerTimestamps.foreach { ts => + val keyRow = if (ts < 0) { + dataToKeyRowWithRangeScan(ts.toLong, "a") + } else dataToKeyRowWithRangeScan(ts.toLong, ts.toString) + val valueRow = dataToValueRow(1) + store.put(keyRow, valueRow, cfName) + assert(valueRowToData(store.get(keyRow, cfName)) === 1) + } + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + // remove + store.remove(dataToKeyRowWithRangeScan(1L, 1L.toString), cfName) + timerTimestamps = timerTimestamps.filter(_ != 1L) + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + // prefix scan + val keyRow = dataToPrefixKeyRowWithRangeScan(1L) + assert(store.prefixScan(keyRow, cfName).toSeq.length + == timerTimestamps.filter(_ == 1L).length) + + store.commit() + } + } } override def newStoreProvider(): RocksDBStateStoreProvider = { 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 6a6867fbb5523..7acedf5af384d 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 @@ -16,38 +16,39 @@ */ package org.apache.spark.sql.execution.streaming.state - -import java.io.{File, IOException} +//scalastyle:off +//import java.io.{File, IOException} +import java.io.File import java.net.URI import java.util -import java.util.UUID +//import java.util.UUID import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable -import scala.jdk.CollectionConverters._ +//import scala.jdk.CollectionConverters._ import scala.util.Random -import org.apache.commons.io.FileUtils +//import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} -import org.scalatest.concurrent.Eventually._ -import org.scalatest.time.SpanSugar._ +//import org.scalatest.concurrent.Eventually._ +//import org.scalatest.time.SpanSugar._ -import org.apache.spark._ -import org.apache.spark.LocalSparkContext._ -import org.apache.spark.sql.SparkSession +//import org.apache.spark._ +//import org.apache.spark.LocalSparkContext._ +//import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorSuite.withCoordinatorRef -import org.apache.spark.sql.functions.count +//import org.apache.spark.sql.execution.streaming._ +//import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorSuite.withCoordinatorRef +//import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils - +// scalastyle:on class FakeStateStoreProviderWithMaintenanceError extends StateStoreProvider { import FakeStateStoreProviderWithMaintenanceError._ private var id: StateStoreId = null @@ -90,7 +91,7 @@ private object FakeStateStoreProviderWithMaintenanceError { class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] with BeforeAndAfter { import StateStoreTestsHelper._ - import StateStoreCoordinatorSuite._ + // import StateStoreCoordinatorSuite._ before { StateStore.stop() @@ -101,7 +102,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] StateStore.stop() require(!StateStore.isMaintenanceRunning) } - +/* test("retaining only two latest versions when MAX_BATCHES_TO_RETAIN_IN_MEMORY set to 2") { tryWithProviderResource( newStoreProvider(minDeltasForSnapshot = 10, numOfVersToRetainInMemory = 2)) { provider => @@ -910,7 +911,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] assertCacheHitAndMiss(reloadedStoreV2.metrics, expectedCacheHitCount = 0, expectedCacheMissCount = 2) } - } + } */ override def newStoreProvider(): HDFSBackedStateStoreProvider = { newStoreProvider(opId = Random.nextInt(), partition = 0) @@ -1053,7 +1054,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] protected val keySchema: StructType = StateStoreTestsHelper.keySchema protected val valueSchema: StructType = StateStoreTestsHelper.valueSchema - +/* testWithAllCodec("get, put, remove, commit, and all data iterator") { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => // Verify state before starting a new set of updates @@ -1624,7 +1625,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] // Shouldn't throw StateStoreProvider.validateStateRowFormat( keyRow, keySchema, valueRow, keySchema, storeConf) - } + } */ /** Return a new provider with a random id */ def newStoreProvider(): ProviderClass From d96e164f908144a629c4d0961325fbc136038295 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 27 Jun 2024 11:27:59 -0700 Subject: [PATCH 02/23] refactor test suites, have some failing tests --- .../state/HDFSBackedStateStoreProvider.scala | 1 + .../streaming/state/RocksDBStateEncoder.scala | 1 + .../state/RocksDBStateStoreProvider.scala | 69 +++- .../streaming/state/StateStore.scala | 1 + .../streaming/state/MemoryStateStore.scala | 1 + .../RocksDBStateStoreIntegrationSuite.scala | 10 +- .../state/RocksDBStateStoreSuite.scala | 316 +++++++----------- .../streaming/state/RocksDBSuite.scala | 124 ++++--- 8 files changed, 247 insertions(+), 276 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 543cd74c489d0..ca3e4ba1ab1ba 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -126,6 +126,7 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, + useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit = { throw StateStoreErrors.multipleColumnFamiliesNotSupported(providerName) } 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 92d9af8eac3f8..5a21d80cce99f 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 @@ -33,6 +33,7 @@ sealed trait RocksDBKeyStateEncoder { def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] + // TODO try change less of the API for key encoder def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = throw new IllegalArgumentException(s"Unsupported encodePrefixKey with colFamilyId arg") 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 80c1e79b3ff46..b129037e08f71 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 @@ -28,7 +28,7 @@ import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors -import org.apache.spark.sql.internal.SQLConf.STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED +// import org.apache.spark.sql.internal.SQLConf.STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.Platform import org.apache.spark.util.Utils @@ -57,18 +57,21 @@ private[sql] class RocksDBStateStoreProvider valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, + useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit = { verify(colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME, s"Failed to create column family with reserved_name=$colFamilyName") verify(useColumnFamilies, "Column families are not supported in this store") - rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) - val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf) - useVirtualColumnFamily = sparkConf.get(STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED) + useVirtualColumnFamily = useVirtualColFamily + println("I am here inside using rocksdb state store provider: " + useVirtualColumnFamily) if (useVirtualColumnFamily) { // if use virtual column family, then use default col family, no need to create new - rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) + // TODO avoid Id duplication + println("I am here inside using virtual col family") colFamilyToLongMap.putIfAbsent(colFamilyName, scala.util.Random.nextLong()) + } else { + rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) } keyValueEncoderMap.putIfAbsent(colFamilyName, @@ -76,10 +79,13 @@ private[sql] class RocksDBStateStoreProvider RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) } + // TODO verify and throw error if colFamilyToLongMap does not have id + // TODO check rocksDB.get function verify works for VCF override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) val value = if (useVirtualColumnFamily) { + println("I am here inside use virtual col family") kvEncoder._2.decodeValue( rocksDB.get(kvEncoder._1.encodeKey(key, colFamilyToLongMap.get(colFamilyName)))) @@ -115,8 +121,14 @@ private[sql] class RocksDBStateStoreProvider verify(valueEncoder.supportsMultipleValuesPerKey, "valuesIterator requires a encoder " + "that supports multiple values for a single key.") - val encodedKey = rocksDB.get(keyEncoder.encodeKey(key), colFamilyName) - valueEncoder.decodeValues(encodedKey) + + val encodedValues = if (useVirtualColumnFamily) { + println("I am here inside virtual col family valuesIterator") + rocksDB.get(keyEncoder.encodeKey(key, colFamilyToLongMap.get(colFamilyName))) + } else { + rocksDB.get(keyEncoder.encodeKey(key), colFamilyName) + } + valueEncoder.decodeValues(encodedValues) } override def merge(key: UnsafeRow, value: UnsafeRow, @@ -129,7 +141,14 @@ private[sql] class RocksDBStateStoreProvider " which supports multiple values for a single key") verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - rocksDB.merge(keyEncoder.encodeKey(key), valueEncoder.encodeValue(value), colFamilyName) + val (encodedKey, physicalColFamilyName) = if (useVirtualColumnFamily) { + println("I am here inside virtual col family merge") + (keyEncoder.encodeKey(key, colFamilyToLongMap.get(colFamilyName)), + StateStore.DEFAULT_COL_FAMILY_NAME) + } else { + (keyEncoder.encodeKey(key), colFamilyName) + } + rocksDB.merge(encodedKey, valueEncoder.encodeValue(value), physicalColFamilyName) } override def put(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = { @@ -158,14 +177,6 @@ private[sql] class RocksDBStateStoreProvider } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { - // How to avoid memcpy here - def getIdBytes(id: Long): Array[Byte] = { - // Long is fixed to be 8 bytes - val encodedBytes = new Array[Byte](8) - Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) - encodedBytes - } - val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() if (useVirtualColumnFamily) { @@ -317,11 +328,33 @@ private[sql] class RocksDBStateStoreProvider /** Return the [[RocksDB]] instance in this store. This is exposed mainly for testing. */ def dbInstance(): RocksDB = rocksDB + // TODO How to avoid memcpy here + private def getIdBytes(id: Long): Array[Byte] = { + // Long is fixed to be 8 bytes + val encodedBytes = new Array[Byte](8) + Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) + encodedBytes + } + /** Remove column family if exists */ override def removeColFamilyIfExists(colFamilyName: String): Boolean = { verify(useColumnFamilies, "Column families are not supported in this store") - val result = rocksDB.removeColFamilyIfExists(colFamilyName) - keyValueEncoderMap.remove(colFamilyName) + val result = if (!useVirtualColumnFamily) { + rocksDB.removeColFamilyIfExists(colFamilyName) + } else { + // TODO more efficient way to do remove col family? + val idPrefix = getIdBytes( + colFamilyToLongMap.get(colFamilyName) + ) + var colFamilyExists = false + rocksDB.prefixScan(idPrefix).foreach { kv => + logInfo("I am here inside remove col family") + colFamilyExists = true + rocksDB.remove(kv.key) + } + colFamilyExists + } + colFamilyToLongMap.remove(colFamilyName) 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 8c2170abe3116..3d56113f87afb 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 @@ -131,6 +131,7 @@ trait StateStore extends ReadStateStore { valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, + useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala index 6a476635a6dbe..fa56256634699 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala @@ -36,6 +36,7 @@ class MemoryStateStore extends StateStore() { valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, + useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit = { throw StateStoreErrors.multipleColumnFamiliesNotSupported("MemoryStateStoreProvider") } 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 384a31f5aa7e6..aff7a55a96f96 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 @@ -35,7 +35,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest import testImplicits._ testWithColumnFamilies("RocksDBStateStore", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => withTempDir { dir => val input = MemoryStream[Int] val conf = Map(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> @@ -62,7 +62,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest testWithColumnFamilies("SPARK-36236: query progress contains only the " + s"expected RocksDB store custom metrics", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => // fails if any new custom metrics are added to remind the author of API changes import testImplicits._ @@ -123,7 +123,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest } testWithColumnFamilies("SPARK-36519: store RocksDB format version in the checkpoint", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => withSQLConf( SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName) { withTempDir { dir => @@ -161,7 +161,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest } testWithColumnFamilies("SPARK-36519: RocksDB format version can be set by the SQL conf", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => withSQLConf( SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, SQLConf.STATE_STORE_ROCKSDB_FORMAT_VERSION.key -> "100") { @@ -183,7 +183,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest testWithColumnFamilies("SPARK-37224: numRowsTotal = 0 when " + s"trackTotalNumberOfRows is turned off", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => withTempDir { dir => withSQLConf( (SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName), 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 2428a504acaee..68891b60094fd 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 @@ -17,28 +17,28 @@ package org.apache.spark.sql.execution.streaming.state // scalastyle:off -//import java.util.UUID +import java.util.UUID -//import scala.collection.immutable +import scala.collection.immutable import scala.util.Random import org.apache.hadoop.conf.Configuration import org.scalatest.BeforeAndAfter +import org.apache.spark.{SparkConf, SparkUnsupportedOperationException} -//import org.apache.spark.{SparkConf, SparkUnsupportedOperationException} import org.apache.spark.io.CompressionCodec -//import org.apache.spark.sql.LocalSparkSession.withSparkSession -//import org.apache.spark.sql.SparkSession -//import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} +import org.apache.spark.sql.LocalSparkSession.withSparkSession +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.catalyst.util.quietly -//import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo +import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSparkSession import org.apache.spark.sql.types._ import org.apache.spark.tags.ExtendedSQLTest -//import org.apache.spark.unsafe.Platform -//import org.apache.spark.unsafe.types.UTF8String -//import org.apache.spark.util.Utils +import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.Utils // scalastyle:on @ExtendedSQLTest @@ -58,9 +58,10 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } import StateStoreTestsHelper._ -/* + testWithColumnFamilies(s"version encoding", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, _) => import RocksDBStateStoreProvider._ tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => @@ -129,7 +130,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb file manager metrics exposed", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, _) => import RocksDBStateStoreProvider._ def getCustomMetric(metrics: StateStoreMetrics, customMetric: StateStoreCustomMetric): Long = { @@ -164,7 +166,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - invalid num columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, _) => // zero ordering cols val ex1 = intercept[SparkUnsupportedOperationException] { tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, @@ -203,7 +206,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - variable sized columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, _) => val keySchemaWithVariableSizeCols: StructType = StructType( Seq(StructField("key1", StringType, false), StructField("key2", StringType, false))) @@ -226,7 +230,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - variable size data types unsupported", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, _) => val keySchemaWithSomeUnsupportedTypeCols: StructType = StructType(Seq( StructField("key1", StringType, false), StructField("key2", IntegerType, false), @@ -266,7 +271,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - null type columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, _) => val keySchemaWithNullTypeCols: StructType = StructType( Seq(StructField("key1", NullType, false), StructField("key2", StringType, false))) @@ -289,7 +295,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - fixed size non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), @@ -301,7 +308,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, @@ -342,7 +350,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan - variable size non-ordering columns with " + "double type values are supported", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( Seq(StructField("key1", DoubleType, false), @@ -357,7 +366,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0))) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0)), + useVirtualColFamily = virtualColFamilyEnabled) } // Verify that the sort ordering here is as follows: @@ -397,7 +407,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - variable size non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), @@ -408,7 +419,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, 90L, 1L, 2L, 8L, 3L, 35L, 6L, 9L, 5L, @@ -451,7 +463,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( Seq(StructField("key1", LongType, false), @@ -468,7 +481,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps = Seq((931L, 10), (8000L, 40), (452300L, 1), (4200L, 68), (90L, 2000), @@ -494,7 +508,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan multiple non-contiguous ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled ) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( Seq( StructField("ordering-1", LongType, false), @@ -524,7 +539,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, rangeScanOrdinals) + RangeKeyScanStateEncoderSpec(testSchema, rangeScanOrdinals), + useVirtualColFamily = virtualColFamilyEnabled ) } @@ -585,7 +601,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns with null values in first ordering column", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( Seq(StructField("key1", LongType, true), @@ -602,7 +619,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps = Seq((931L, 10), (null, 40), (452300L, 1), @@ -658,7 +676,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store1.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps1 = Seq((null, 3), (null, 1), (null, 32), @@ -685,7 +704,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns with null values in second ordering column", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( Seq(StructField("key1", LongType, true), @@ -702,7 +722,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps = Seq((931L, 10), (40L, null), (452300L, 1), @@ -738,7 +759,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan byte ordering column - variable size " + s"non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( Seq(StructField("key1", ByteType, false), @@ -755,7 +777,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps: Seq[(Byte, Int)] = Seq((0x33, 10), (0x1A, 40), (0x1F, 1), (0x01, 68), @@ -781,7 +804,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - ordering cols and key schema cols are same", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => // use the same schema as value schema for single col key schema tryWithProviderResource(newStoreProvider(valueSchema, @@ -791,7 +815,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, valueSchema, valueSchema, - RangeKeyScanStateEncoderSpec(valueSchema, Seq(0))) + RangeKeyScanStateEncoderSpec(valueSchema, Seq(0)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps = Seq(931, 8000, 452300, 4200, @@ -823,7 +848,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - with prefix scan", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, virtualColFamilyEnabled) => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), @@ -834,7 +860,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), + useVirtualColFamily = virtualColFamilyEnabled) } val timerTimestamps = Seq(931L, -1331L, 8000L, 1L, -244L, -8350L, -55L) @@ -860,176 +887,66 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb key and value schema encoders for column families", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val testColFamily = "testState" - - tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => - val store = provider.getStore(0) - if (colFamiliesEnabled) { - store.createColFamilyIfAbsent(testColFamily, - keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) - val keyRow1 = dataToKeyRow("a", 0) - val valueRow1 = dataToValueRow(1) - store.put(keyRow1, valueRow1, colFamilyName = testColFamily) - assert(valueRowToData(store.get(keyRow1, colFamilyName = testColFamily)) === 1) - store.remove(keyRow1, colFamilyName = testColFamily) - assert(store.get(keyRow1, colFamilyName = testColFamily) === null) - } - val keyRow2 = dataToKeyRow("b", 0) - val valueRow2 = dataToValueRow(2) - store.put(keyRow2, valueRow2) - assert(valueRowToData(store.get(keyRow2)) === 2) - store.remove(keyRow2) - assert(store.get(keyRow2) === null) - } - } - - test("validate rocksdb values iterator correctness") { - withSQLConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key -> "1") { - tryWithProviderResource(newStoreProvider(useColumnFamilies = true, - useMultipleValuesPerKey = true)) { provider => - val store = provider.getStore(0) - // Verify state after updating - put(store, "a", 0, 1) - - val iterator0 = store.valuesIterator(dataToKeyRow("a", 0)) - - assert(iterator0.hasNext) - assert(valueRowToData(iterator0.next()) === 1) - assert(!iterator0.hasNext) - - merge(store, "a", 0, 2) - merge(store, "a", 0, 3) - - val iterator1 = store.valuesIterator(dataToKeyRow("a", 0)) - - (1 to 3).map { i => - assert(iterator1.hasNext) - assert(valueRowToData(iterator1.next()) === i) - } - - assert(!iterator1.hasNext) - - remove(store, _._1 == "a") - val iterator2 = store.valuesIterator(dataToKeyRow("a", 0)) - assert(!iterator2.hasNext) - - assert(get(store, "a", 0).isEmpty) - } - } - } */ - - test("test virtual col family (noPrefix) - " + - "put & get & remove, iterator, prefixScan, remove col family") { - withSQLConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled" -> "true") { - tryWithProviderResource(newStoreProvider(keySchema, - NoPrefixKeyStateEncoderSpec(keySchema), true)) { provider => + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { + (colFamiliesEnabled, virtualColFamilyEnabled) => + val testColFamily = "testState" + tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => val store = provider.getStore(0) - - // use non-default col family if column families are enabled - val cfName = "testColFamily" - store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, - NoPrefixKeyStateEncoderSpec(keySchema)) - - var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, - -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) - // put & get, iterator - timerTimestamps.foreach { ts => - val keyRow = if (ts < 0) { - dataToKeyRow("a", ts.toInt) - } else dataToKeyRow(ts.toString, ts.toInt) - val valueRow = dataToValueRow(1) - store.put(keyRow, valueRow, cfName) - assert(valueRowToData(store.get(keyRow, cfName)) === 1) + if (colFamiliesEnabled) { + store.createColFamilyIfAbsent(testColFamily, + keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema), + useVirtualColFamily = virtualColFamilyEnabled) + val keyRow1 = dataToKeyRow("a", 0) + val valueRow1 = dataToValueRow(1) + store.put(keyRow1, valueRow1, colFamilyName = testColFamily) + assert(valueRowToData(store.get(keyRow1, colFamilyName = testColFamily)) === 1) + store.remove(keyRow1, colFamilyName = testColFamily) + assert(store.get(keyRow1, colFamilyName = testColFamily) === null) } - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) - - // remove - store.remove(dataToKeyRow(1L.toString, 1.toInt), cfName) - timerTimestamps = timerTimestamps.filter(_ != 1L) - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) - - store.commit() - } + val keyRow2 = dataToKeyRow("b", 0) + val valueRow2 = dataToValueRow(2) + store.put(keyRow2, valueRow2) + assert(valueRowToData(store.get(keyRow2)) === 2) + store.remove(keyRow2) + assert(store.get(keyRow2) === null) } } - test("test virtual col family (prefix) - " + - "put & get & remove, iterator, prefixScan, remove col family") { - withSQLConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled" -> "true") { - tryWithProviderResource(newStoreProvider(keySchema, - PrefixKeyScanStateEncoderSpec(keySchema, 1), true)) { provider => - val store = provider.getStore(0) + Seq(true, false).foreach { virtualColFamilyEnabled => + test(s"validate rocksdb values iterator correctness " + + s"with virtualColFamilyEnabled=$virtualColFamilyEnabled") { + withSQLConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key -> "1") { + tryWithProviderResource(newStoreProvider(useColumnFamilies = true, + useMultipleValuesPerKey = true, + useVirtualColFamily = virtualColFamilyEnabled)) { provider => + val store = provider.getStore(0) + // Verify state after updating + put(store, "a", 0, 1) - // use non-default col family if column families are enabled - val cfName = "testColFamily" - store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, - PrefixKeyScanStateEncoderSpec(keySchema, 1)) - - var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, - -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) - // put & get, iterator - timerTimestamps.foreach { ts => - val keyRow = if (ts < 0) { - dataToKeyRow("a", ts.toInt) - } else dataToKeyRow(ts.toString, ts.toInt) - val valueRow = dataToValueRow(1) - store.put(keyRow, valueRow, cfName) - assert(valueRowToData(store.get(keyRow, cfName)) === 1) - } - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + val iterator0 = store.valuesIterator(dataToKeyRow("a", 0)) - // remove - store.remove(dataToKeyRow(1L.toString, 1.toInt), cfName) - timerTimestamps = timerTimestamps.filter(_ != 1L) - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + assert(iterator0.hasNext) + assert(valueRowToData(iterator0.next()) === 1) + assert(!iterator0.hasNext) - // prefix scan - val keyRow = dataToPrefixKeyRow("a") - assert(store.prefixScan(keyRow, cfName).toSeq.length - == timerTimestamps.filter(_ < 0).length) + merge(store, "a", 0, 2) + merge(store, "a", 0, 3) - store.commit() - } - } - } - - test("test virtual col family (rangeScan) - " + - "put & get & remove, iterator, prefixScan, remove col family") { - withSQLConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled" -> "true") { - tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), true)) { provider => - val store = provider.getStore(0) + val iterator1 = store.valuesIterator(dataToKeyRow("a", 0)) - // use non-default col family if column families are enabled - val cfName = "testColFamily" - store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) - - var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, - -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) - // put & get, iterator - timerTimestamps.foreach { ts => - val keyRow = if (ts < 0) { - dataToKeyRowWithRangeScan(ts.toLong, "a") - } else dataToKeyRowWithRangeScan(ts.toLong, ts.toString) - val valueRow = dataToValueRow(1) - store.put(keyRow, valueRow, cfName) - assert(valueRowToData(store.get(keyRow, cfName)) === 1) - } - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + (1 to 3).map { i => + assert(iterator1.hasNext) + assert(valueRowToData(iterator1.next()) === i) + } - // remove - store.remove(dataToKeyRowWithRangeScan(1L, 1L.toString), cfName) - timerTimestamps = timerTimestamps.filter(_ != 1L) - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + assert(!iterator1.hasNext) - // prefix scan - val keyRow = dataToPrefixKeyRowWithRangeScan(1L) - assert(store.prefixScan(keyRow, cfName).toSeq.length - == timerTimestamps.filter(_ == 1L).length) + remove(store, _._1 == "a") + val iterator2 = store.valuesIterator(dataToKeyRow("a", 0)) + assert(!iterator2.hasNext) - store.commit() + assert(get(store, "a", 0).isEmpty) + } } } } @@ -1055,11 +972,13 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } def newStoreProvider(useColumnFamilies: Boolean, - useMultipleValuesPerKey: Boolean): RocksDBStateStoreProvider = { + useMultipleValuesPerKey: Boolean, + useVirtualColFamily: Boolean): RocksDBStateStoreProvider = { newStoreProvider(StateStoreId(newDir(), Random.nextInt(), 0), NoPrefixKeyStateEncoderSpec(keySchema), useColumnFamilies = useColumnFamilies, - useMultipleValuesPerKey = useMultipleValuesPerKey + useMultipleValuesPerKey = useMultipleValuesPerKey, + useVirtualColFamily = useVirtualColFamily ) } @@ -1084,7 +1003,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid sqlConf: Option[SQLConf] = None, conf: Configuration = new Configuration, useColumnFamilies: Boolean = false, - useMultipleValuesPerKey: Boolean = false): RocksDBStateStoreProvider = { + useMultipleValuesPerKey: Boolean = false, + useVirtualColFamily: Boolean = false): RocksDBStateStoreProvider = { val provider = new RocksDBStateStoreProvider() provider.init( storeId, 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 6086fd43846f6..981569596d77b 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 @@ -127,34 +127,48 @@ trait AlsoTestWithChangelogCheckpointingEnabled def testWithColumnFamilies( testName: String, testMode: TestMode, + alsoTestWithVirtualColFamily: Boolean, testTags: Tag*) - (testBody: Boolean => Any): Unit = { + (testBody: (Boolean, Boolean) => Any): Unit = { Seq(true, false).foreach { colFamiliesEnabled => - testMode match { - case TestWithChangelogCheckpointingEnabled => - testWithChangelogCheckpointingEnabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { - testBody(colFamiliesEnabled) - } + val virtualColFamilySeq = + if (alsoTestWithVirtualColFamily) Seq(true, false) else Seq(false) + + virtualColFamilySeq.foreach { virtualColFamilyEnabled => + val testNameSuffix = + if (alsoTestWithVirtualColFamily) { + s" & virtualColFamilyEnabled=$virtualColFamilyEnabled" + } else "" + testMode match { + case TestWithChangelogCheckpointingEnabled => + testWithChangelogCheckpointingEnabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, + testTags: _*) { + testBody(colFamiliesEnabled, virtualColFamilyEnabled) + } - case TestWithChangelogCheckpointingDisabled => - testWithChangelogCheckpointingDisabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { - testBody(colFamiliesEnabled) - } + case TestWithChangelogCheckpointingDisabled => + testWithChangelogCheckpointingDisabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, + testTags: _*) { + testBody(colFamiliesEnabled, virtualColFamilyEnabled) + } - case TestWithBothChangelogCheckpointingEnabledAndDisabled => - testWithChangelogCheckpointingEnabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { - testBody(colFamiliesEnabled) - } - testWithChangelogCheckpointingDisabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { - testBody(colFamiliesEnabled) - } + case TestWithBothChangelogCheckpointingEnabledAndDisabled => + testWithChangelogCheckpointingEnabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, + testTags: _*) { + testBody(colFamiliesEnabled, virtualColFamilyEnabled) + } + testWithChangelogCheckpointingDisabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, + testTags: _*) { + testBody(colFamiliesEnabled, virtualColFamilyEnabled) + } - case _ => - throw new IllegalArgumentException(s"Unknown test mode: $testMode") + case _ => + throw new IllegalArgumentException(s"Unknown test mode: $testMode") + } } } } @@ -167,7 +181,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies( "RocksDB: check changelog and snapshot version", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 1) new File(remoteDir).delete() // to make sure that the directory gets created @@ -190,8 +204,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: load version that doesn't exist", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { - colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => val provider = new RocksDBStateStoreProvider() var ex = intercept[SparkException] { provider.getStore(-1) @@ -228,7 +241,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies( "RocksDB: purge changelog and snapshots", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val conf = dbConf.copy(enableChangelogCheckpointing = true, @@ -270,7 +283,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies( "RocksDB: minDeltasForSnapshot", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val conf = dbConf.copy(enableChangelogCheckpointing = true, minDeltasForSnapshot = 3) @@ -312,7 +325,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("SPARK-45419: Do not reuse SST files" + " in different RocksDB instances", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 0, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created @@ -344,7 +357,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // an existing checkpoint without changelog. testWithColumnFamilies( "RocksDB: changelog checkpointing backward compatibility", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val disableChangelogCheckpointingConf = @@ -407,7 +420,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // an existing checkpoint with changelog. testWithColumnFamilies( "RocksDB: changelog checkpointing forward compatibility", - TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => + TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val enableChangelogCheckpointingConf = @@ -459,7 +472,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: compression conf", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -475,7 +488,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: get, put, iterator, commit, load", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => def testOps(compactOnCommit: Boolean): Unit = { val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -545,7 +558,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: column family creation with invalid names", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -582,7 +595,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: column family creation with reserved chars", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -651,7 +664,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: operations on absent column family", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -687,7 +700,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies(s"RocksDB: get, put, iterator, commit, load " + s"with multiple column families", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val colFamily1: String = "abc" @@ -803,7 +816,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: handle commit failures and aborts", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => val hadoopConf = new Configuration() hadoopConf.set( SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, @@ -990,7 +1003,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("RocksDBFileManager: create init dfs directory with " + s"unknown number of keys", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") try { val verificationDir = Utils.createTempDir().getAbsolutePath @@ -1080,7 +1093,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("RocksDBFileManager: delete orphan files", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath // Use 2 file managers here to emulate concurrent execution @@ -1160,7 +1173,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("RocksDBFileManager: don't delete orphan files " + s"when there is only 1 version", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath val fileManager = new RocksDBFileManager( @@ -1215,7 +1228,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("RocksDBFileManager: upload only new immutable files", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath val verificationDir = Utils.createTempDir().getAbsolutePath // local dir to load checkpoints @@ -1305,7 +1318,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("RocksDBFileManager: error writing [version].zip " + s"cancels the output stream", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => quietly { val hadoopConf = new Configuration() hadoopConf.set( @@ -1323,7 +1336,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("disallow concurrent updates to the same RocksDB instance", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => quietly { withDB( Utils.createTempDir().toString, @@ -1393,7 +1406,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("ensure concurrent access lock is released after Spark task completes", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => RocksDBSuite.withSingletonDB { // Load a RocksDB instance, that is, get a lock inside a task and then fail quietly { @@ -1411,7 +1424,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("checkpoint metadata serde roundtrip", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => // expect read metadata error when metadata uses unsupported version withTempDir { dir => val file2 = new File(dir, "json") @@ -1459,7 +1472,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("SPARK-36236: reset RocksDB metrics whenever a new version is loaded", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => def verifyMetrics(putCount: Long, getCount: Long, iterCountPositive: Boolean = false, metrics: RocksDBMetrics): Unit = { assert(metrics.nativeOpsHistograms("put").count === putCount, "invalid put count") @@ -1558,7 +1571,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("-1", "100", "1000").foreach { maxOpenFiles => testWithColumnFamilies(s"SPARK-39781: adding valid max_open_files=$maxOpenFiles " + "config property for RocksDB state store instance should succeed", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => withTempDir { dir => val sqlConf = SQLConf.get.clone() sqlConf.setConfString("spark.sql.streaming.stateStore.rocksdb.maxOpenFiles", maxOpenFiles) @@ -1581,7 +1594,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("test", "true").foreach { maxOpenFiles => testWithColumnFamilies(s"SPARK-39781: adding invalid max_open_files=$maxOpenFiles config " + "property for RocksDB state store instance should fail", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => withTempDir { dir => val ex = intercept[IllegalArgumentException] { val sqlConf = SQLConf.get.clone() @@ -1609,7 +1622,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("16", "32", "64").foreach {writeBufferSizeMB => testWithColumnFamilies(s"SPARK-42819: configure memtable memory usage with " + s"maxWriteBufferNumber=$maxWriteBufferNumber and writeBufferSize=$writeBufferSizeMB", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => withTempDir { dir => val sqlConf = new SQLConf sqlConf.setConfString("spark.sql.streaming.stateStore.rocksdb.maxWriteBufferNumber", @@ -1634,7 +1647,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("Verify that fallocate is allowed by default", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => val sqlConf = new SQLConf val dbConf = RocksDBConf(StateStoreConf(sqlConf)) assert(dbConf.allowFAllocate == true) @@ -1642,7 +1655,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared /** RocksDB memory management tests for bounded memory usage */ testWithColumnFamilies("Memory mgmt - invalid config", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => withTempDir { dir => try { RocksDBMemoryManager.resetWriteBufferManagerAndCache @@ -1681,7 +1694,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("true", "false").foreach { boundedMemoryUsage => testWithColumnFamilies(s"Memory mgmt - Cache reuse for RocksDB " + s"with boundedMemoryUsage=$boundedMemoryUsage", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + (colFamiliesEnabled, _) => withTempDir { dir1 => withTempDir { dir2 => try { @@ -1741,7 +1755,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("100", "1000", "100000").foreach { totalMemorySizeMB => testWithColumnFamilies(s"Memory mgmt - valid config " + s"with totalMemorySizeMB=$totalMemorySizeMB", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => withTempDir { dir => try { val sqlConf = new SQLConf @@ -1778,7 +1792,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("SPARK-37224: flipping option 'trackTotalNumberOfRows' during restart", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => withTempDir { dir => val remoteDir = dir.getCanonicalPath From eb7378de0a709dccb92c65798ce9b76f24c4ba49 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 27 Jun 2024 11:51:02 -0700 Subject: [PATCH 03/23] a full working version with full test suites --- .../streaming/state/RocksDBStateEncoder.scala | 51 ++++++++++++------- .../state/RocksDBStateStoreProvider.scala | 5 -- .../state/RocksDBStateStoreSuite.scala | 20 ++++---- 3 files changed, 44 insertions(+), 32 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 5a21d80cce99f..12a74a85094ff 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 @@ -30,18 +30,11 @@ import org.apache.spark.unsafe.Platform sealed trait RocksDBKeyStateEncoder { def supportPrefixKeyScan: Boolean - def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] - - // TODO try change less of the API for key encoder - def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = - throw new IllegalArgumentException(s"Unsupported encodePrefixKey with colFamilyId arg") - + // TODO try change less of the API for key encoder? + def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] def encodeKey(row: UnsafeRow): Array[Byte] - - def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = - throw new IllegalArgumentException(s"Unsupported encodeKey with colFamilyId arg") - + def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] def decodeKey(keyBytes: Array[Byte], hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow } @@ -615,10 +608,11 @@ class RangeKeyScanStateEncoder( private def decodeKeyWithColFamilyPrefix( keyBytes: Array[Byte], hasVirtualColFamilyPrefix: Boolean): UnsafeRow = { - val prefixKeyEncodedLen = Platform.getInt(keyBytes, Platform.BYTE_ARRAY_OFFSET) - val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + val prefixKeyEncodedLen = Platform.getInt( + keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) + val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) @@ -656,15 +650,33 @@ class RangeKeyScanStateEncoder( decodeKeyWithColFamilyPrefix(keyBytes, hasVirtualColFamilyPrefix) } - override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { + private def encodePrefixKeyWithColFamilyPrefix( + prefixKey: UnsafeRow, + hasVirtualColFamilyPrefix: Boolean = false, + colFamilyId: Long = -1L): Array[Byte] = { + val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) - val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4) - Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET, rangeScanKeyEncoded.length) - Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefix, - Platform.BYTE_ARRAY_OFFSET + 4, rangeScanKeyEncoded.length) + val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) + + if (hasVirtualColFamilyPrefix) { + Platform.putLong(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + } + Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + rangeScanKeyEncoded.length) + Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, + prefix, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, rangeScanKeyEncoded.length) prefix } + override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { + encodePrefixKeyWithColFamilyPrefix(prefixKey, false) + } + + override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = { + encodePrefixKeyWithColFamilyPrefix(prefixKey, true, colFamilyId) + } + override def supportPrefixKeyScan: Boolean = true } @@ -732,6 +744,11 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { throw new IllegalStateException("This encoder doesn't support prefix key!") } + + override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = { + throw new IllegalArgumentException("This encoder doesn't support prefix key encoding" + + "with column family Id!") + } } /** 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 b129037e08f71..6d11f7b1c94c9 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 @@ -64,11 +64,9 @@ private[sql] class RocksDBStateStoreProvider verify(useColumnFamilies, "Column families are not supported in this store") useVirtualColumnFamily = useVirtualColFamily - println("I am here inside using rocksdb state store provider: " + useVirtualColumnFamily) if (useVirtualColumnFamily) { // if use virtual column family, then use default col family, no need to create new // TODO avoid Id duplication - println("I am here inside using virtual col family") colFamilyToLongMap.putIfAbsent(colFamilyName, scala.util.Random.nextLong()) } else { rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) @@ -85,7 +83,6 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) val value = if (useVirtualColumnFamily) { - println("I am here inside use virtual col family") kvEncoder._2.decodeValue( rocksDB.get(kvEncoder._1.encodeKey(key, colFamilyToLongMap.get(colFamilyName)))) @@ -123,7 +120,6 @@ private[sql] class RocksDBStateStoreProvider "that supports multiple values for a single key.") val encodedValues = if (useVirtualColumnFamily) { - println("I am here inside virtual col family valuesIterator") rocksDB.get(keyEncoder.encodeKey(key, colFamilyToLongMap.get(colFamilyName))) } else { rocksDB.get(keyEncoder.encodeKey(key), colFamilyName) @@ -142,7 +138,6 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") val (encodedKey, physicalColFamilyName) = if (useVirtualColumnFamily) { - println("I am here inside virtual col family merge") (keyEncoder.encodeKey(key, colFamilyToLongMap.get(colFamilyName)), StateStore.DEFAULT_COL_FAMILY_NAME) } else { 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 68891b60094fd..4d75e22898350 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 @@ -295,7 +295,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - fixed size non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, @@ -350,7 +350,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan - variable size non-ordering columns with " + "double type values are supported", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( @@ -407,7 +407,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - variable size non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, @@ -463,7 +463,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( @@ -508,7 +508,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan multiple non-contiguous ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( Seq( @@ -601,7 +601,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns with null values in first ordering column", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( @@ -704,7 +704,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns with null values in second ordering column", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( @@ -759,7 +759,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan byte ordering column - variable size " + s"non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => val testSchema: StructType = StructType( @@ -804,7 +804,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - ordering cols and key schema cols are same", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => // use the same schema as value schema for single col key schema @@ -848,7 +848,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - with prefix scan", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { (colFamiliesEnabled, virtualColFamilyEnabled) => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, From 2f44aaf57a629937aa8d90e755b9b633b33c858a Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 27 Jun 2024 12:46:58 -0700 Subject: [PATCH 04/23] need to fix removeColFamilyIfExists test suite --- .../state/RocksDBStateStoreProvider.scala | 2 +- .../state/RocksDBStateStoreSuite.scala | 109 ++++++++++++++++++ .../streaming/state/StateStoreSuite.scala | 4 +- 3 files changed, 112 insertions(+), 3 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 6d11f7b1c94c9..0207d1baffbfc 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 @@ -28,7 +28,6 @@ import org.apache.spark.internal.{Logging, MDC} import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors -// import org.apache.spark.sql.internal.SQLConf.STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.Platform import org.apache.spark.util.Utils @@ -79,6 +78,7 @@ private[sql] class RocksDBStateStoreProvider // TODO verify and throw error if colFamilyToLongMap does not have id // TODO check rocksDB.get function verify works for VCF + // TODO verify with changelog checkpoint override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) 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 4d75e22898350..7353de05a803f 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 @@ -912,6 +912,74 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } } + // TODO This is the same as test("prefix scan") in StateStoreSuite without codec + testWithColumnFamilies("rocksdb prefix key encoder", + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { + (colFamiliesEnabled, virtualColFamilyEnabled) => + tryWithProviderResource(newStoreProvider( + keySchema, PrefixKeyScanStateEncoderSpec(keySchema, 1), + colFamiliesEnabled)) { provider => + var store = provider.getStore(0) + + val testColFamily = "testState" + if (colFamiliesEnabled) { + store.createColFamilyIfAbsent(testColFamily, + keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema), + useVirtualColFamily = virtualColFamilyEnabled) + } + + def putCompositeKeys(keys: Seq[(String, Int)]): Unit = { + val randomizedKeys = scala.util.Random.shuffle(keys.toList) + randomizedKeys.foreach { case (key1, key2) => + put(store, key1, key2, key2) + } + } + + def verifyScan(key1: Seq[String], key2: Seq[Int]): Unit = { + key1.foreach { k1 => + val keyValueSet = store.prefixScan(dataToPrefixKeyRow(k1)).map { pair => + rowPairToDataPair(pair.withRows(pair.key.copy(), pair.value.copy())) + }.toSet + + assert(keyValueSet === key2.map(k2 => ((k1, k2), k2)).toSet) + } + } + + val key1AtVersion0 = Seq("a", "b", "c") + val key2AtVersion0 = Seq(1, 2, 3) + val keysAtVersion0 = for (k1 <- key1AtVersion0; k2 <- key2AtVersion0) yield (k1, k2) + + putCompositeKeys(keysAtVersion0) + verifyScan(key1AtVersion0, key2AtVersion0) + + assert(store.prefixScan(dataToPrefixKeyRow("non-exist")).isEmpty) + + // committing and loading the version 1 (the version being committed) + store.commit() + store = provider.getStore(1) + + // before putting the new key-value pairs, verify prefix scan works for existing keys + verifyScan(key1AtVersion0, key2AtVersion0) + + val key1AtVersion1 = Seq("c", "d") + val key2AtVersion1 = Seq(4, 5, 6) + val keysAtVersion1 = for (k1 <- key1AtVersion1; k2 <- key2AtVersion1) yield (k1, k2) + + // put a new key-value pairs, and verify that prefix scan reflects the changes + putCompositeKeys(keysAtVersion1) + verifyScan(Seq("c"), Seq(1, 2, 3, 4, 5, 6)) + verifyScan(Seq("d"), Seq(4, 5, 6)) + + // aborting and loading the version 1 again (keysAtVersion1 should be rolled back) + store.abort() + store = provider.getStore(1) + + // prefix scan should not reflect the uncommitted changes + verifyScan(key1AtVersion0, key2AtVersion0) + verifyScan(Seq("d"), Seq.empty) + } + } + Seq(true, false).foreach { virtualColFamilyEnabled => test(s"validate rocksdb values iterator correctness " + s"with virtualColFamilyEnabled=$virtualColFamilyEnabled") { @@ -951,6 +1019,47 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } } + Seq(true, false).foreach { virtualColFamilyEnabled => + test(s"validate rocksdb removeColFamilyIfExists correctness " + + s"with virtualColFamilyEnabled=$virtualColFamilyEnabled") { + Seq( + // NoPrefixKeyStateEncoderSpec(keySchema) + PrefixKeyScanStateEncoderSpec(keySchema, 1) + // RangeKeyScanStateEncoderSpec(keySchema, Seq(1)) + ).foreach { keyEncoder => + tryWithProviderResource(newStoreProvider(keySchema, keyEncoder, true)) { provider => + val store = provider.getStore(0) + + val cfName = "testColFamily" + store.createColFamilyIfAbsent(cfName, + keySchema, valueSchema, keyEncoder, + useVirtualColFamily = virtualColFamilyEnabled) + + // put some test data into state store + val timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, + -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) + timerTimestamps.foreach { ts => + val keyRow = dataToKeyRow(ts.toString, ts.toInt) + val valueRow = dataToValueRow(1) + store.put(keyRow, valueRow, cfName) + } + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + store.removeColFamilyIfExists(cfName) + + val e = intercept[Exception] { + store.iterator(cfName) + } + checkError( + exception = e.asInstanceOf[SparkUnsupportedOperationException], + errorClass = "STATEFUL_PROCESSOR_CANNOT_REINITIALIZE_STATE_ON_KEY", + sqlState = Some("42802"), + parameters = Map("groupingKey" -> "init_1") + ) + }} + } + } + override def newStoreProvider(): RocksDBStateStoreProvider = { newStoreProvider(StateStoreId(newDir(), Random.nextInt(), 0)) } 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 7acedf5af384d..25875a10c39bc 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 @@ -1109,7 +1109,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] === Set(("b", 0) -> 2)) } } - } + } */ testWithAllCodec("prefix scan") { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(keySchema, PrefixKeyScanStateEncoderSpec(keySchema, 1), @@ -1170,7 +1170,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] verifyScan(Seq("d"), Seq.empty) } } - +/* testWithAllCodec(s"numKeys metrics") { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => // Verify state before starting a new set of updates From bb4e2bfcbd3656f3a943526a28a4cd828219ac49 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 27 Jun 2024 13:36:15 -0700 Subject: [PATCH 05/23] fix all suites, add validation suites for noprefix & prefix --- .../streaming/state/RocksDBStateEncoder.scala | 4 +- .../state/RocksDBStateStoreProvider.scala | 1 - .../state/RocksDBStateStoreSuite.scala | 117 ++++++++---------- 3 files changed, 54 insertions(+), 68 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 12a74a85094ff..7499bce63a2f8 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 @@ -708,11 +708,11 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) STATE_ENCODING_NUM_VERSION_BYTES + 8) Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) - Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET, STATE_ENCODING_VERSION) + Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + 8, STATE_ENCODING_VERSION) // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. Platform.copyMemory( bytesToEncode, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + 8, bytesToEncode.length) encodedBytes } 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 0207d1baffbfc..9ace82ebee0e6 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 @@ -343,7 +343,6 @@ private[sql] class RocksDBStateStoreProvider ) var colFamilyExists = false rocksDB.prefixScan(idPrefix).foreach { kv => - logInfo("I am here inside remove col family") colFamilyExists = true rocksDB.remove(kv.key) } 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 7353de05a803f..753fe5d0c2c78 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 @@ -912,72 +912,52 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } } - // TODO This is the same as test("prefix scan") in StateStoreSuite without codec - testWithColumnFamilies("rocksdb prefix key encoder", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => - tryWithProviderResource(newStoreProvider( - keySchema, PrefixKeyScanStateEncoderSpec(keySchema, 1), + Seq( + NoPrefixKeyStateEncoderSpec(keySchema), PrefixKeyScanStateEncoderSpec(keySchema, 1) + ).foreach { keyEncoder => + testWithColumnFamilies(s"validate rocksdb " + + s"${keyEncoder.getClass.toString.split('.').last} correctness", + TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { + (colFamiliesEnabled, virtualColFamilyEnabled) => + tryWithProviderResource(newStoreProvider(keySchema, keyEncoder, colFamiliesEnabled)) { provider => - var store = provider.getStore(0) + val store = provider.getStore(0) - val testColFamily = "testState" + val cfName = if (colFamiliesEnabled) "testColFamily" else "default" if (colFamiliesEnabled) { - store.createColFamilyIfAbsent(testColFamily, - keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema), + store.createColFamilyIfAbsent(cfName, + keySchema, valueSchema, keyEncoder, useVirtualColFamily = virtualColFamilyEnabled) } - def putCompositeKeys(keys: Seq[(String, Int)]): Unit = { - val randomizedKeys = scala.util.Random.shuffle(keys.toList) - randomizedKeys.foreach { case (key1, key2) => - put(store, key1, key2, key2) - } + var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, + -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) + // put & get, iterator + timerTimestamps.foreach { ts => + val keyRow = if (ts < 0) { + dataToKeyRow("a", ts.toInt) + } else dataToKeyRow(ts.toString, ts.toInt) + val valueRow = dataToValueRow(1) + store.put(keyRow, valueRow, cfName) + assert(valueRowToData(store.get(keyRow, cfName)) === 1) } + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) - def verifyScan(key1: Seq[String], key2: Seq[Int]): Unit = { - key1.foreach { k1 => - val keyValueSet = store.prefixScan(dataToPrefixKeyRow(k1)).map { pair => - rowPairToDataPair(pair.withRows(pair.key.copy(), pair.value.copy())) - }.toSet + // remove + store.remove(dataToKeyRow(1L.toString, 1.toInt), cfName) + timerTimestamps = timerTimestamps.filter(_ != 1L) + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) - assert(keyValueSet === key2.map(k2 => ((k1, k2), k2)).toSet) - } + // prefix scan + if (!keyEncoder.getClass.toString.contains("No")) { + val keyRow = dataToPrefixKeyRow("a") + assert(store.prefixScan(keyRow, cfName).toSeq.length + == timerTimestamps.filter(_ < 0).length) } - val key1AtVersion0 = Seq("a", "b", "c") - val key2AtVersion0 = Seq(1, 2, 3) - val keysAtVersion0 = for (k1 <- key1AtVersion0; k2 <- key2AtVersion0) yield (k1, k2) - - putCompositeKeys(keysAtVersion0) - verifyScan(key1AtVersion0, key2AtVersion0) - - assert(store.prefixScan(dataToPrefixKeyRow("non-exist")).isEmpty) - - // committing and loading the version 1 (the version being committed) store.commit() - store = provider.getStore(1) - - // before putting the new key-value pairs, verify prefix scan works for existing keys - verifyScan(key1AtVersion0, key2AtVersion0) - - val key1AtVersion1 = Seq("c", "d") - val key2AtVersion1 = Seq(4, 5, 6) - val keysAtVersion1 = for (k1 <- key1AtVersion1; k2 <- key2AtVersion1) yield (k1, k2) - - // put a new key-value pairs, and verify that prefix scan reflects the changes - putCompositeKeys(keysAtVersion1) - verifyScan(Seq("c"), Seq(1, 2, 3, 4, 5, 6)) - verifyScan(Seq("d"), Seq(4, 5, 6)) - - // aborting and loading the version 1 again (keysAtVersion1 should be rolled back) - store.abort() - store = provider.getStore(1) - - // prefix scan should not reflect the uncommitted changes - verifyScan(key1AtVersion0, key2AtVersion0) - verifyScan(Seq("d"), Seq.empty) } + } } Seq(true, false).foreach { virtualColFamilyEnabled => @@ -1023,9 +1003,9 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid test(s"validate rocksdb removeColFamilyIfExists correctness " + s"with virtualColFamilyEnabled=$virtualColFamilyEnabled") { Seq( - // NoPrefixKeyStateEncoderSpec(keySchema) - PrefixKeyScanStateEncoderSpec(keySchema, 1) - // RangeKeyScanStateEncoderSpec(keySchema, Seq(1)) + NoPrefixKeyStateEncoderSpec(keySchema), + PrefixKeyScanStateEncoderSpec(keySchema, 1), + RangeKeyScanStateEncoderSpec(keySchema, Seq(1)) ).foreach { keyEncoder => tryWithProviderResource(newStoreProvider(keySchema, keyEncoder, true)) { provider => val store = provider.getStore(0) @@ -1045,17 +1025,24 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) - store.removeColFamilyIfExists(cfName) + // assert col family existence + assert(store.removeColFamilyIfExists(cfName)) + + // TODO eliminate behavior difference + if (virtualColFamilyEnabled) { + assert(!store.iterator(cfName).hasNext) + } else { + val e = intercept[Exception] { + store.iterator(cfName) + } - val e = intercept[Exception] { - store.iterator(cfName) + checkError( + exception = e.asInstanceOf[StateStoreUnsupportedOperationOnMissingColumnFamily], + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + sqlState = Some("42802"), + parameters = Map("operationType" -> "iterator", "colFamilyName" -> cfName) + ) } - checkError( - exception = e.asInstanceOf[SparkUnsupportedOperationException], - errorClass = "STATEFUL_PROCESSOR_CANNOT_REINITIALIZE_STATE_ON_KEY", - sqlState = Some("42802"), - parameters = Map("groupingKey" -> "init_1") - ) }} } } From febcb0702481853ebd66c5e4331befcee1455276 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 27 Jun 2024 14:26:05 -0700 Subject: [PATCH 06/23] few TODOs, integration to be added --- .../streaming/state/RocksDBStateStoreProvider.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 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 9ace82ebee0e6..54ebaf27d6ff5 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 @@ -65,8 +65,9 @@ private[sql] class RocksDBStateStoreProvider useVirtualColumnFamily = useVirtualColFamily if (useVirtualColumnFamily) { // if use virtual column family, then use default col family, no need to create new - // TODO avoid Id duplication + // TODO how to efficiently guarantee there isn't any value conflict for different key colFamilyToLongMap.putIfAbsent(colFamilyName, scala.util.Random.nextLong()) + } else { rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) } @@ -76,12 +77,11 @@ private[sql] class RocksDBStateStoreProvider RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) } - // TODO verify and throw error if colFamilyToLongMap does not have id - // TODO check rocksDB.get function verify works for VCF // TODO verify with changelog checkpoint override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) + val value = if (useVirtualColumnFamily) { kvEncoder._2.decodeValue( rocksDB.get(kvEncoder._1.encodeKey(key, @@ -362,6 +362,9 @@ private[sql] class RocksDBStateStoreProvider storeConf: StateStoreConf, hadoopConf: Configuration, useMultipleValuesPerKey: Boolean = false): Unit = { + // TODO should we propagate useVirtualColFamily as a param + // TODO how to expose the virtual col family to the users + // - cluster config? operator-wise? stateStore-wise? this.stateStoreId_ = stateStoreId this.keySchema = keySchema this.valueSchema = valueSchema From 08561914a9496b1ebc58e1e9c5f5369b8b952615 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 27 Jun 2024 14:28:50 -0700 Subject: [PATCH 07/23] scala style --- .../state/RocksDBStateStoreSuite.scala | 3 +- .../streaming/state/StateStoreSuite.scala | 41 +++++++++---------- 2 files changed, 21 insertions(+), 23 deletions(-) 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 753fe5d0c2c78..8cfb9672de253 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 @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.execution.streaming.state -// scalastyle:off + import java.util.UUID import scala.collection.immutable @@ -39,7 +39,6 @@ import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils -// scalastyle:on @ExtendedSQLTest class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvider] 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 25875a10c39bc..9c99f7df6b9db 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 @@ -16,39 +16,38 @@ */ package org.apache.spark.sql.execution.streaming.state -//scalastyle:off -//import java.io.{File, IOException} -import java.io.File + +import java.io.{File, IOException} import java.net.URI import java.util -//import java.util.UUID +import java.util.UUID import java.util.concurrent.atomic.AtomicBoolean import scala.collection.mutable -//import scala.jdk.CollectionConverters._ +import scala.jdk.CollectionConverters._ import scala.util.Random -//import org.apache.commons.io.FileUtils +import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} -//import org.scalatest.concurrent.Eventually._ -//import org.scalatest.time.SpanSugar._ +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ -//import org.apache.spark._ -//import org.apache.spark.LocalSparkContext._ -//import org.apache.spark.sql.SparkSession +import org.apache.spark._ +import org.apache.spark.LocalSparkContext._ +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.util.quietly -//import org.apache.spark.sql.execution.streaming._ -//import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorSuite.withCoordinatorRef -//import org.apache.spark.sql.functions.count +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorSuite.withCoordinatorRef +import org.apache.spark.sql.functions.count import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.tags.ExtendedSQLTest import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils -// scalastyle:on + class FakeStateStoreProviderWithMaintenanceError extends StateStoreProvider { import FakeStateStoreProviderWithMaintenanceError._ private var id: StateStoreId = null @@ -102,7 +101,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] StateStore.stop() require(!StateStore.isMaintenanceRunning) } -/* + test("retaining only two latest versions when MAX_BATCHES_TO_RETAIN_IN_MEMORY set to 2") { tryWithProviderResource( newStoreProvider(minDeltasForSnapshot = 10, numOfVersToRetainInMemory = 2)) { provider => @@ -911,7 +910,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] assertCacheHitAndMiss(reloadedStoreV2.metrics, expectedCacheHitCount = 0, expectedCacheMissCount = 2) } - } */ + } override def newStoreProvider(): HDFSBackedStateStoreProvider = { newStoreProvider(opId = Random.nextInt(), partition = 0) @@ -1054,7 +1053,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] protected val keySchema: StructType = StateStoreTestsHelper.keySchema protected val valueSchema: StructType = StateStoreTestsHelper.valueSchema -/* + testWithAllCodec("get, put, remove, commit, and all data iterator") { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => // Verify state before starting a new set of updates @@ -1109,7 +1108,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] === Set(("b", 0) -> 2)) } } - } */ + } testWithAllCodec("prefix scan") { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(keySchema, PrefixKeyScanStateEncoderSpec(keySchema, 1), @@ -1170,7 +1169,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] verifyScan(Seq("d"), Seq.empty) } } -/* + testWithAllCodec(s"numKeys metrics") { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => // Verify state before starting a new set of updates @@ -1625,7 +1624,7 @@ abstract class StateStoreSuiteBase[ProviderClass <: StateStoreProvider] // Shouldn't throw StateStoreProvider.validateStateRowFormat( keyRow, keySchema, valueRow, keySchema, storeConf) - } */ + } /** Return a new provider with a random id */ def newStoreProvider(): ProviderClass From f906369cc7f399de0cb3ada1c38bbbc86a8f11a1 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Thu, 27 Jun 2024 14:30:53 -0700 Subject: [PATCH 08/23] scala style --- .../sql/execution/streaming/state/RocksDBStateStoreSuite.scala | 2 +- .../spark/sql/execution/streaming/state/StateStoreSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) 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 8cfb9672de253..25d57849a2b93 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 @@ -24,8 +24,8 @@ import scala.util.Random import org.apache.hadoop.conf.Configuration import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkUnsupportedOperationException} +import org.apache.spark.{SparkConf, SparkUnsupportedOperationException} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.LocalSparkSession.withSparkSession import org.apache.spark.sql.SparkSession 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 9c99f7df6b9db..6a6867fbb5523 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 @@ -90,7 +90,7 @@ private object FakeStateStoreProviderWithMaintenanceError { class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] with BeforeAndAfter { import StateStoreTestsHelper._ - // import StateStoreCoordinatorSuite._ + import StateStoreCoordinatorSuite._ before { StateStore.stop() From 36a5247167d56f783966705108ecf246fcf5d0ae Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 1 Jul 2024 13:34:37 -0700 Subject: [PATCH 09/23] use 2 bytes as prefix instead of 8 --- .../apache/spark/sql/internal/SQLConf.scala | 9 --- .../streaming/state/RocksDBStateEncoder.scala | 73 +++++++++++-------- .../state/RocksDBStateStoreProvider.scala | 33 +++++---- 3 files changed, 59 insertions(+), 56 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index faee5a5890a05..06e0c6eda5896 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -2168,15 +2168,6 @@ object SQLConf { .checkValue(v => Set(1, 2).contains(v), "Valid versions are 1 and 2") .createWithDefault(2) - val STREAMING_ROCKSDB_VIRTUAL_COL_FAMILY_ENABLED = - buildConf("spark.databricks.streaming.rocksDBVirtualColFamily.enabled") - .internal() - .doc("Whether structured streaming use virtual column family. Currently this is " + - "only supported with TransformWithState operator.") - .version("4.0.0") - .booleanConf - .createWithDefault(false) - val STREAMING_STOP_ACTIVE_RUN_ON_RESTART = buildConf("spark.sql.streaming.stopActiveRunOnRestart") .doc("Running multiple runs of the same streaming query concurrently is not supported. " + 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 7499bce63a2f8..c622424f317aa 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 @@ -24,7 +24,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.internal.Logging 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.state.RocksDBStateStoreProvider.{STATE_ENCODING_NUM_VERSION_BYTES, STATE_ENCODING_VERSION} +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.types._ import org.apache.spark.unsafe.Platform @@ -32,9 +32,9 @@ sealed trait RocksDBKeyStateEncoder { def supportPrefixKeyScan: Boolean def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] // TODO try change less of the API for key encoder? - def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] + def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] def encodeKey(row: UnsafeRow): Array[Byte] - def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] + def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] def decodeKey(keyBytes: Array[Byte], hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow } @@ -153,15 +153,16 @@ class PrefixKeyScanStateEncoder( private def encodeKeyWithColFamilyPrefix( row: UnsafeRow, hasVirtualColFamilyPrefix: Boolean, - colFamilyId: Long = -1L): Array[Byte] = { + colFamilyId: Short = 0): Array[Byte] = { val prefixKeyEncoded = encodeUnsafeRow(extractPrefixKey(row)) val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) - val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + val offSetForColFamilyPrefix = + if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val encodedBytes = new Array[Byte](prefixKeyEncoded.length + remainingEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) } Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -183,15 +184,15 @@ class PrefixKeyScanStateEncoder( encodeKeyWithColFamilyPrefix(row, false) } - override def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = { + override def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] = { encodeKeyWithColFamilyPrefix(row, true, colFamilyId) } private def decodeKeyWithColFamilyPrefix( keyBytes: Array[Byte], hasVirtualColFamilyPrefix: Boolean): UnsafeRow = { - val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 - + val offSetForColFamilyPrefix = + if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val prefixKeyEncodedLen = Platform.getInt( keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) @@ -201,7 +202,8 @@ class PrefixKeyScanStateEncoder( Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes - val remainingKeyEncodedLen = keyBytes.length - 4 - prefixKeyEncodedLen + val remainingKeyEncodedLen = keyBytes.length - 4 - prefixKeyEncodedLen - + offSetForColFamilyPrefix val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + @@ -229,14 +231,15 @@ class PrefixKeyScanStateEncoder( private def encodePrefixKeyWithColFamilyPrefix( prefixKey: UnsafeRow, hasVirtualColFamilyPrefix: Boolean = false, - colFamilyId: Long = -1L): Array[Byte] = { - val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + colFamilyId: Short = 0): Array[Byte] = { + val offSetForColFamilyPrefix = + if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val prefixKeyEncoded = encodeUnsafeRow(prefixKey) val prefix = new Array[Byte]( prefixKeyEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putLong(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) } Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, prefixKeyEncoded.length) @@ -250,7 +253,7 @@ class PrefixKeyScanStateEncoder( encodePrefixKeyWithColFamilyPrefix(prefixKey) } - override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] = { encodePrefixKeyWithColFamilyPrefix(prefixKey, true, colFamilyId) } @@ -550,12 +553,13 @@ class RangeKeyScanStateEncoder( private def encodeKeyWithColFamilyPrefix( row: UnsafeRow, hasVirtualColFamilyPrefix: Boolean = false, - colFamilyId: Long = -1L): Array[Byte] = { + colFamilyId: Short = 0): Array[Byte] = { // This prefix key has the columns specified by orderingOrdinals val prefixKey = extractPrefixKey(row) val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) - val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + val offSetForColFamilyPrefix = + if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val result = if (orderingOrdinals.length < keySchema.length) { val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) @@ -563,7 +567,7 @@ class RangeKeyScanStateEncoder( remainingEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) } Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -584,7 +588,7 @@ class RangeKeyScanStateEncoder( val encodedBytes = new Array[Byte]( rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) } Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -601,14 +605,15 @@ class RangeKeyScanStateEncoder( encodeKeyWithColFamilyPrefix(row, false) } - override def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = { + override def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] = { encodeKeyWithColFamilyPrefix(row, true, colFamilyId) } private def decodeKeyWithColFamilyPrefix( keyBytes: Array[Byte], hasVirtualColFamilyPrefix: Boolean): UnsafeRow = { - val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + val offSetForColFamilyPrefix = + if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val prefixKeyEncodedLen = Platform.getInt( keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) @@ -653,14 +658,15 @@ class RangeKeyScanStateEncoder( private def encodePrefixKeyWithColFamilyPrefix( prefixKey: UnsafeRow, hasVirtualColFamilyPrefix: Boolean = false, - colFamilyId: Long = -1L): Array[Byte] = { - val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) 8 else 0 + colFamilyId: Short = 0): Array[Byte] = { + val offSetForColFamilyPrefix = + if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putLong(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) } Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, rangeScanKeyEncoded.length) @@ -673,7 +679,7 @@ class RangeKeyScanStateEncoder( encodePrefixKeyWithColFamilyPrefix(prefixKey, false) } - override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] = { encodePrefixKeyWithColFamilyPrefix(prefixKey, true, colFamilyId) } @@ -702,17 +708,19 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) override def encodeKey(row: UnsafeRow): Array[Byte] = encodeUnsafeRow(row) - override def encodeKey(row: UnsafeRow, colFamilyId: Long): Array[Byte] = { + override def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] = { val bytesToEncode = row.getBytes val encodedBytes = new Array[Byte](bytesToEncode.length + - STATE_ENCODING_NUM_VERSION_BYTES + 8) + STATE_ENCODING_NUM_VERSION_BYTES + VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) - Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + 8, STATE_ENCODING_VERSION) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES, + STATE_ENCODING_VERSION) // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. Platform.copyMemory( bytesToEncode, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + 8, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + + VIRTUAL_COL_FAMILY_PREFIX_BYTES, bytesToEncode.length) encodedBytes } @@ -730,8 +738,9 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. keyRow.pointTo( keyBytes, - Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + 8, - keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - 8) + Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + + VIRTUAL_COL_FAMILY_PREFIX_BYTES, + keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - VIRTUAL_COL_FAMILY_PREFIX_BYTES) keyRow } else { null @@ -745,7 +754,7 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) throw new IllegalStateException("This encoder doesn't support prefix key!") } - override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Long): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] = { throw new IllegalArgumentException("This encoder doesn't support prefix key encoding" + "with column family Id!") } 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 54ebaf27d6ff5..5611fac8525b1 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 @@ -66,7 +66,10 @@ private[sql] class RocksDBStateStoreProvider if (useVirtualColumnFamily) { // if use virtual column family, then use default col family, no need to create new // TODO how to efficiently guarantee there isn't any value conflict for different key - colFamilyToLongMap.putIfAbsent(colFamilyName, scala.util.Random.nextLong()) + def getNextRandShort: Short = { + (scala.util.Random.nextInt(Short.MaxValue - Short.MinValue + 1) + Short.MinValue).toShort + } + colFamilyToIdMap.putIfAbsent(colFamilyName, getNextRandShort) } else { rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) @@ -85,7 +88,7 @@ private[sql] class RocksDBStateStoreProvider val value = if (useVirtualColumnFamily) { kvEncoder._2.decodeValue( rocksDB.get(kvEncoder._1.encodeKey(key, - colFamilyToLongMap.get(colFamilyName)))) + colFamilyToIdMap.get(colFamilyName)))) } else { kvEncoder._2.decodeValue( rocksDB.get(kvEncoder._1.encodeKey(key), colFamilyName)) @@ -120,7 +123,7 @@ private[sql] class RocksDBStateStoreProvider "that supports multiple values for a single key.") val encodedValues = if (useVirtualColumnFamily) { - rocksDB.get(keyEncoder.encodeKey(key, colFamilyToLongMap.get(colFamilyName))) + rocksDB.get(keyEncoder.encodeKey(key, colFamilyToIdMap.get(colFamilyName))) } else { rocksDB.get(keyEncoder.encodeKey(key), colFamilyName) } @@ -138,7 +141,7 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") val (encodedKey, physicalColFamilyName) = if (useVirtualColumnFamily) { - (keyEncoder.encodeKey(key, colFamilyToLongMap.get(colFamilyName)), + (keyEncoder.encodeKey(key, colFamilyToIdMap.get(colFamilyName)), StateStore.DEFAULT_COL_FAMILY_NAME) } else { (keyEncoder.encodeKey(key), colFamilyName) @@ -152,7 +155,7 @@ private[sql] class RocksDBStateStoreProvider require(value != null, "Cannot put a null value") val kvEncoder = keyValueEncoderMap.get(colFamilyName) if (useVirtualColumnFamily) { - rocksDB.put(kvEncoder._1.encodeKey(key, colFamilyToLongMap.get(colFamilyName)), + rocksDB.put(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName)), kvEncoder._2.encodeValue(value)) } else { rocksDB.put(kvEncoder._1.encodeKey(key), @@ -165,7 +168,7 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) if (useVirtualColumnFamily) { - rocksDB.remove(kvEncoder._1.encodeKey(key, colFamilyToLongMap.get(colFamilyName))) + rocksDB.remove(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName))) } else { rocksDB.remove(kvEncoder._1.encodeKey(key), colFamilyName) } @@ -175,7 +178,7 @@ private[sql] class RocksDBStateStoreProvider val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() if (useVirtualColumnFamily) { - val cfId: Long = colFamilyToLongMap.get(colFamilyName) + val cfId: Short = colFamilyToIdMap.get(colFamilyName) rocksDB.prefixScan(getIdBytes(cfId)).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), kvEncoder._2.decodeValue(kv.value)) @@ -208,7 +211,7 @@ private[sql] class RocksDBStateStoreProvider val rowPair = new UnsafeRowPair() if (useVirtualColumnFamily) { - val prefix = kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToLongMap.get(colFamilyName)) + val prefix = kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToIdMap.get(colFamilyName)) rocksDB.prefixScan(prefix).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), @@ -324,10 +327,9 @@ private[sql] class RocksDBStateStoreProvider def dbInstance(): RocksDB = rocksDB // TODO How to avoid memcpy here - private def getIdBytes(id: Long): Array[Byte] = { - // Long is fixed to be 8 bytes - val encodedBytes = new Array[Byte](8) - Platform.putLong(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) + private def getIdBytes(id: Short): Array[Byte] = { + val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) encodedBytes } @@ -339,7 +341,7 @@ private[sql] class RocksDBStateStoreProvider } else { // TODO more efficient way to do remove col family? val idPrefix = getIdBytes( - colFamilyToLongMap.get(colFamilyName) + colFamilyToIdMap.get(colFamilyName) ) var colFamilyExists = false rocksDB.prefixScan(idPrefix).foreach { kv => @@ -348,7 +350,7 @@ private[sql] class RocksDBStateStoreProvider } colFamilyExists } - colFamilyToLongMap.remove(colFamilyName) + colFamilyToIdMap.remove(colFamilyName) result } } @@ -454,7 +456,7 @@ private[sql] class RocksDBStateStoreProvider private val keyValueEncoderMap = new java.util.concurrent.ConcurrentHashMap[String, (RocksDBKeyStateEncoder, RocksDBValueStateEncoder)] - private val colFamilyToLongMap = new java.util.concurrent.ConcurrentHashMap[String, Long] + private val colFamilyToIdMap = new java.util.concurrent.ConcurrentHashMap[String, Short] private var useVirtualColumnFamily: Boolean = false @@ -467,6 +469,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 // Native operation latencies report as latency in microseconds // as SQLMetrics support millis. Convert the value to millis From 4c0af0e5f121bb17bb298e90f9dbb94505ffd226 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Tue, 2 Jul 2024 10:22:37 -0700 Subject: [PATCH 10/23] unused codes in RocksDB --- .../state/HDFSBackedStateStoreProvider.scala | 1 - .../state/RocksDBStateStoreProvider.scala | 111 ++----- .../streaming/state/StateStore.scala | 4 +- .../streaming/state/MemoryStateStore.scala | 1 - .../RocksDBStateStoreIntegrationSuite.scala | 10 +- .../state/RocksDBStateStoreSuite.scala | 284 ++++++++---------- .../streaming/state/RocksDBSuite.scala | 124 ++++---- 7 files changed, 212 insertions(+), 323 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index ca3e4ba1ab1ba..543cd74c489d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -126,7 +126,6 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, - useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit = { throw StateStoreErrors.multipleColumnFamiliesNotSupported(providerName) } 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 5611fac8525b1..d88ca08bff8d2 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 @@ -56,43 +56,29 @@ private[sql] class RocksDBStateStoreProvider valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, - useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit = { verify(colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME, s"Failed to create column family with reserved_name=$colFamilyName") verify(useColumnFamilies, "Column families are not supported in this store") - useVirtualColumnFamily = useVirtualColFamily - if (useVirtualColumnFamily) { - // if use virtual column family, then use default col family, no need to create new - // TODO how to efficiently guarantee there isn't any value conflict for different key - def getNextRandShort: Short = { - (scala.util.Random.nextInt(Short.MaxValue - Short.MinValue + 1) + Short.MinValue).toShort - } - colFamilyToIdMap.putIfAbsent(colFamilyName, getNextRandShort) - - } else { - rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) + // TODO how to efficiently guarantee there isn't any value conflict for different key + def getNextRandShort: Short = { + (scala.util.Random.nextInt(Short.MaxValue - Short.MinValue + 1) + Short.MinValue).toShort } + colFamilyToIdMap.putIfAbsent(colFamilyName, getNextRandShort) keyValueEncoderMap.putIfAbsent(colFamilyName, (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec), RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) } - // TODO verify with changelog checkpoint override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val value = if (useVirtualColumnFamily) { - kvEncoder._2.decodeValue( + val value = kvEncoder._2.decodeValue( rocksDB.get(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName)))) - } else { - kvEncoder._2.decodeValue( - rocksDB.get(kvEncoder._1.encodeKey(key), colFamilyName)) - } if (!isValidated && value != null && !useColumnFamilies) { StateStoreProvider.validateStateRowFormat( @@ -122,11 +108,8 @@ private[sql] class RocksDBStateStoreProvider verify(valueEncoder.supportsMultipleValuesPerKey, "valuesIterator requires a encoder " + "that supports multiple values for a single key.") - val encodedValues = if (useVirtualColumnFamily) { + val encodedValues = rocksDB.get(keyEncoder.encodeKey(key, colFamilyToIdMap.get(colFamilyName))) - } else { - rocksDB.get(keyEncoder.encodeKey(key), colFamilyName) - } valueEncoder.decodeValues(encodedValues) } @@ -140,12 +123,10 @@ private[sql] class RocksDBStateStoreProvider " which supports multiple values for a single key") verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - val (encodedKey, physicalColFamilyName) = if (useVirtualColumnFamily) { + val (encodedKey, physicalColFamilyName) = (keyEncoder.encodeKey(key, colFamilyToIdMap.get(colFamilyName)), StateStore.DEFAULT_COL_FAMILY_NAME) - } else { - (keyEncoder.encodeKey(key), colFamilyName) - } + rocksDB.merge(encodedKey, valueEncoder.encodeValue(value), physicalColFamilyName) } @@ -154,52 +135,31 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot put a null value") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - if (useVirtualColumnFamily) { - rocksDB.put(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName)), - kvEncoder._2.encodeValue(value)) - } else { - rocksDB.put(kvEncoder._1.encodeKey(key), - kvEncoder._2.encodeValue(value), colFamilyName) - } + rocksDB.put(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName)), + kvEncoder._2.encodeValue(value)) } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { verify(state == UPDATING, "Cannot remove after already committed or aborted") verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - if (useVirtualColumnFamily) { - rocksDB.remove(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName))) - } else { - rocksDB.remove(kvEncoder._1.encodeKey(key), colFamilyName) - } + rocksDB.remove(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName))) } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() - if (useVirtualColumnFamily) { - val cfId: Short = colFamilyToIdMap.get(colFamilyName) - rocksDB.prefixScan(getIdBytes(cfId)).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), - kvEncoder._2.decodeValue(kv.value)) - if (!isValidated && rowPair.value != null && !useColumnFamilies) { - StateStoreProvider.validateStateRowFormat( - rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) - isValidated = true - } - rowPair - } - } else { - rocksDB.iterator(colFamilyName).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key), - kvEncoder._2.decodeValue(kv.value)) - if (!isValidated && rowPair.value != null && !useColumnFamilies) { - StateStoreProvider.validateStateRowFormat( - rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) - isValidated = true - } - rowPair + + val cfId: Short = colFamilyToIdMap.get(colFamilyName) + rocksDB.prefixScan(getIdBytes(cfId)).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), + kvEncoder._2.decodeValue(kv.value)) + if (!isValidated && rowPair.value != null && !useColumnFamilies) { + StateStoreProvider.validateStateRowFormat( + rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) + isValidated = true } + rowPair } } @@ -210,22 +170,12 @@ private[sql] class RocksDBStateStoreProvider "Prefix scan requires setting prefix key!") val rowPair = new UnsafeRowPair() - if (useVirtualColumnFamily) { - val prefix = kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToIdMap.get(colFamilyName)) - rocksDB.prefixScan(prefix).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), - kvEncoder._2.decodeValue(kv.value)) - rowPair - } - } else { - val prefix = kvEncoder._1.encodePrefixKey(prefixKey) - - rocksDB.prefixScan(prefix, colFamilyName).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key), - kvEncoder._2.decodeValue(kv.value)) - rowPair - } + val prefix = kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToIdMap.get(colFamilyName)) + rocksDB.prefixScan(prefix).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), + kvEncoder._2.decodeValue(kv.value)) + rowPair } } @@ -336,9 +286,7 @@ private[sql] class RocksDBStateStoreProvider /** Remove column family if exists */ override def removeColFamilyIfExists(colFamilyName: String): Boolean = { verify(useColumnFamilies, "Column families are not supported in this store") - val result = if (!useVirtualColumnFamily) { - rocksDB.removeColFamilyIfExists(colFamilyName) - } else { + val result = { // TODO more efficient way to do remove col family? val idPrefix = getIdBytes( colFamilyToIdMap.get(colFamilyName) @@ -364,9 +312,6 @@ private[sql] class RocksDBStateStoreProvider storeConf: StateStoreConf, hadoopConf: Configuration, useMultipleValuesPerKey: Boolean = false): Unit = { - // TODO should we propagate useVirtualColFamily as a param - // TODO how to expose the virtual col family to the users - // - cluster config? operator-wise? stateStore-wise? this.stateStoreId_ = stateStoreId this.keySchema = keySchema this.valueSchema = valueSchema @@ -458,8 +403,6 @@ private[sql] class RocksDBStateStoreProvider private val colFamilyToIdMap = new java.util.concurrent.ConcurrentHashMap[String, Short] - private var useVirtualColumnFamily: Boolean = false - private def verify(condition: => Boolean, msg: String): Unit = { if (!condition) { throw new IllegalStateException(msg) } } 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 3d56113f87afb..831cadf82ff66 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 @@ -131,7 +131,6 @@ trait StateStore extends ReadStateStore { valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, - useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit /** @@ -342,7 +341,8 @@ trait StateStoreProvider { * A value not greater than 0 means the operator doesn't activate prefix * key, and the operator should not call prefixScan method in StateStore. * @param useColumnFamilies Whether the underlying state store uses a single or multiple column - * families + * families; by default we'll use virtual column family if this parameter + * is set to true * @param storeConfs Configurations used by the StateStores * @param hadoopConf Hadoop configuration that could be used by StateStore to save state data * @param useMultipleValuesPerKey Whether the underlying state store needs to support multiple diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala index fa56256634699..6a476635a6dbe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/MemoryStateStore.scala @@ -36,7 +36,6 @@ class MemoryStateStore extends StateStore() { valueSchema: StructType, keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, - useVirtualColFamily: Boolean = false, isInternal: Boolean = false): Unit = { throw StateStoreErrors.multipleColumnFamiliesNotSupported("MemoryStateStoreProvider") } 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 aff7a55a96f96..384a31f5aa7e6 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 @@ -35,7 +35,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest import testImplicits._ testWithColumnFamilies("RocksDBStateStore", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val input = MemoryStream[Int] val conf = Map(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> @@ -62,7 +62,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest testWithColumnFamilies("SPARK-36236: query progress contains only the " + s"expected RocksDB store custom metrics", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => // fails if any new custom metrics are added to remind the author of API changes import testImplicits._ @@ -123,7 +123,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest } testWithColumnFamilies("SPARK-36519: store RocksDB format version in the checkpoint", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withSQLConf( SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName) { withTempDir { dir => @@ -161,7 +161,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest } testWithColumnFamilies("SPARK-36519: RocksDB format version can be set by the SQL conf", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withSQLConf( SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, SQLConf.STATE_STORE_ROCKSDB_FORMAT_VERSION.key -> "100") { @@ -183,7 +183,7 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest testWithColumnFamilies("SPARK-37224: numRowsTotal = 0 when " + s"trackTotalNumberOfRows is turned off", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => withSQLConf( (SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName), 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 25d57849a2b93..fa4cd306a512a 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 @@ -59,8 +59,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid import StateStoreTestsHelper._ testWithColumnFamilies(s"version encoding", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { - (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => import RocksDBStateStoreProvider._ tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => @@ -129,8 +128,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb file manager metrics exposed", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { - (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => import RocksDBStateStoreProvider._ def getCustomMetric(metrics: StateStoreMetrics, customMetric: StateStoreCustomMetric): Long = { @@ -165,8 +163,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - invalid num columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { - (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => // zero ordering cols val ex1 = intercept[SparkUnsupportedOperationException] { tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, @@ -205,8 +202,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - variable sized columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { - (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val keySchemaWithVariableSizeCols: StructType = StructType( Seq(StructField("key1", StringType, false), StructField("key2", StringType, false))) @@ -229,8 +225,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - variable size data types unsupported", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { - (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val keySchemaWithSomeUnsupportedTypeCols: StructType = StructType(Seq( StructField("key1", StringType, false), StructField("key2", IntegerType, false), @@ -270,8 +265,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan validation - null type columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { - (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val keySchemaWithNullTypeCols: StructType = StructType( Seq(StructField("key1", NullType, false), StructField("key2", StringType, false))) @@ -294,8 +288,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - fixed size non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), @@ -307,8 +300,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) } val timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, @@ -349,8 +341,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan - variable size non-ordering columns with " + "double type values are supported", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val testSchema: StructType = StructType( Seq(StructField("key1", DoubleType, false), @@ -365,8 +356,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0))) } // Verify that the sort ordering here is as follows: @@ -406,8 +396,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - variable size non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), @@ -418,8 +407,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) } val timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, 90L, 1L, 2L, 8L, 3L, 35L, 6L, 9L, 5L, @@ -462,8 +450,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val testSchema: StructType = StructType( Seq(StructField("key1", LongType, false), @@ -480,8 +467,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) } val timerTimestamps = Seq((931L, 10), (8000L, 40), (452300L, 1), (4200L, 68), (90L, 2000), @@ -507,8 +493,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan multiple non-contiguous ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled ) { colFamiliesEnabled => val testSchema: StructType = StructType( Seq( StructField("ordering-1", LongType, false), @@ -538,8 +523,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, rangeScanOrdinals), - useVirtualColFamily = virtualColFamilyEnabled + RangeKeyScanStateEncoderSpec(testSchema, rangeScanOrdinals) ) } @@ -600,8 +584,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns with null values in first ordering column", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val testSchema: StructType = StructType( Seq(StructField("key1", LongType, true), @@ -618,8 +601,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) } val timerTimestamps = Seq((931L, 10), (null, 40), (452300L, 1), @@ -675,8 +657,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store1.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) } val timerTimestamps1 = Seq((null, 3), (null, 1), (null, 32), @@ -703,8 +684,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan multiple ordering columns - variable size " + s"non-ordering columns with null values in second ordering column", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val testSchema: StructType = StructType( Seq(StructField("key1", LongType, true), @@ -721,8 +701,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) } val timerTimestamps = Seq((931L, 10), (40L, null), (452300L, 1), @@ -758,8 +737,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid testWithColumnFamilies("rocksdb range scan byte ordering column - variable size " + s"non-ordering columns", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val testSchema: StructType = StructType( Seq(StructField("key1", ByteType, false), @@ -776,8 +754,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, testSchema, valueSchema, - RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(testSchema, Seq(0, 1))) } val timerTimestamps: Seq[(Byte, Int)] = Seq((0x33, 10), (0x1A, 40), (0x1F, 1), (0x01, 68), @@ -803,8 +780,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - ordering cols and key schema cols are same", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => // use the same schema as value schema for single col key schema tryWithProviderResource(newStoreProvider(valueSchema, @@ -814,8 +790,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, valueSchema, valueSchema, - RangeKeyScanStateEncoderSpec(valueSchema, Seq(0)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(valueSchema, Seq(0))) } val timerTimestamps = Seq(931, 8000, 452300, 4200, @@ -847,8 +822,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb range scan - with prefix scan", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => tryWithProviderResource(newStoreProvider(keySchemaWithRangeScan, RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), @@ -859,8 +833,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid if (colFamiliesEnabled) { store.createColFamilyIfAbsent(cfName, keySchemaWithRangeScan, valueSchema, - RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0)), - useVirtualColFamily = virtualColFamilyEnabled) + RangeKeyScanStateEncoderSpec(keySchemaWithRangeScan, Seq(0))) } val timerTimestamps = Seq(931L, -1331L, 8000L, 1L, -244L, -8350L, -55L) @@ -886,133 +859,123 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } testWithColumnFamilies("rocksdb key and value schema encoders for column families", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => - val testColFamily = "testState" - tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => - val store = provider.getStore(0) - if (colFamiliesEnabled) { - store.createColFamilyIfAbsent(testColFamily, - keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema), - useVirtualColFamily = virtualColFamilyEnabled) - val keyRow1 = dataToKeyRow("a", 0) - val valueRow1 = dataToValueRow(1) - store.put(keyRow1, valueRow1, colFamilyName = testColFamily) - assert(valueRowToData(store.get(keyRow1, colFamilyName = testColFamily)) === 1) - store.remove(keyRow1, colFamilyName = testColFamily) - assert(store.get(keyRow1, colFamilyName = testColFamily) === null) - } - val keyRow2 = dataToKeyRow("b", 0) - val valueRow2 = dataToValueRow(2) - store.put(keyRow2, valueRow2) - assert(valueRowToData(store.get(keyRow2)) === 2) - store.remove(keyRow2) - assert(store.get(keyRow2) === null) + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + val testColFamily = "testState" + + tryWithProviderResource(newStoreProvider(colFamiliesEnabled)) { provider => + val store = provider.getStore(0) + if (colFamiliesEnabled) { + store.createColFamilyIfAbsent(testColFamily, + keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) + val keyRow1 = dataToKeyRow("a", 0) + val valueRow1 = dataToValueRow(1) + store.put(keyRow1, valueRow1, colFamilyName = testColFamily) + assert(valueRowToData(store.get(keyRow1, colFamilyName = testColFamily)) === 1) + store.remove(keyRow1, colFamilyName = testColFamily) + assert(store.get(keyRow1, colFamilyName = testColFamily) === null) + } + val keyRow2 = dataToKeyRow("b", 0) + val valueRow2 = dataToValueRow(2) + store.put(keyRow2, valueRow2) + assert(valueRowToData(store.get(keyRow2)) === 2) + store.remove(keyRow2) + assert(store.get(keyRow2) === null) } } - Seq( - NoPrefixKeyStateEncoderSpec(keySchema), PrefixKeyScanStateEncoderSpec(keySchema, 1) - ).foreach { keyEncoder => - testWithColumnFamilies(s"validate rocksdb " + - s"${keyEncoder.getClass.toString.split('.').last} correctness", - TestWithBothChangelogCheckpointingEnabledAndDisabled, true) { - (colFamiliesEnabled, virtualColFamilyEnabled) => - tryWithProviderResource(newStoreProvider(keySchema, keyEncoder, - colFamiliesEnabled)) { provider => + test("validate rocksdb values iterator correctness") { + withSQLConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key -> "1") { + tryWithProviderResource(newStoreProvider(useColumnFamilies = true, + useMultipleValuesPerKey = true)) { provider => val store = provider.getStore(0) + // Verify state after updating + put(store, "a", 0, 1) - val cfName = if (colFamiliesEnabled) "testColFamily" else "default" - if (colFamiliesEnabled) { - store.createColFamilyIfAbsent(cfName, - keySchema, valueSchema, keyEncoder, - useVirtualColFamily = virtualColFamilyEnabled) - } + val iterator0 = store.valuesIterator(dataToKeyRow("a", 0)) - var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, - -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) - // put & get, iterator - timerTimestamps.foreach { ts => - val keyRow = if (ts < 0) { - dataToKeyRow("a", ts.toInt) - } else dataToKeyRow(ts.toString, ts.toInt) - val valueRow = dataToValueRow(1) - store.put(keyRow, valueRow, cfName) - assert(valueRowToData(store.get(keyRow, cfName)) === 1) - } - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + assert(iterator0.hasNext) + assert(valueRowToData(iterator0.next()) === 1) + assert(!iterator0.hasNext) - // remove - store.remove(dataToKeyRow(1L.toString, 1.toInt), cfName) - timerTimestamps = timerTimestamps.filter(_ != 1L) - assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + merge(store, "a", 0, 2) + merge(store, "a", 0, 3) - // prefix scan - if (!keyEncoder.getClass.toString.contains("No")) { - val keyRow = dataToPrefixKeyRow("a") - assert(store.prefixScan(keyRow, cfName).toSeq.length - == timerTimestamps.filter(_ < 0).length) + val iterator1 = store.valuesIterator(dataToKeyRow("a", 0)) + + (1 to 3).map { i => + assert(iterator1.hasNext) + assert(valueRowToData(iterator1.next()) === i) } - store.commit() + assert(!iterator1.hasNext) + + remove(store, _._1 == "a") + val iterator2 = store.valuesIterator(dataToKeyRow("a", 0)) + assert(!iterator2.hasNext) + + assert(get(store, "a", 0).isEmpty) } } } - Seq(true, false).foreach { virtualColFamilyEnabled => - test(s"validate rocksdb values iterator correctness " + - s"with virtualColFamilyEnabled=$virtualColFamilyEnabled") { - withSQLConf(SQLConf.STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT.key -> "1") { - tryWithProviderResource(newStoreProvider(useColumnFamilies = true, - useMultipleValuesPerKey = true, - useVirtualColFamily = virtualColFamilyEnabled)) { provider => + Seq( + NoPrefixKeyStateEncoderSpec(keySchema), PrefixKeyScanStateEncoderSpec(keySchema, 1) + ).foreach { keyEncoder => + testWithColumnFamilies(s"validate rocksdb " + + s"${keyEncoder.getClass.toString.split('.').last} correctness", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + tryWithProviderResource(newStoreProvider(keySchema, keyEncoder, + colFamiliesEnabled)) { provider => val store = provider.getStore(0) - // Verify state after updating - put(store, "a", 0, 1) - - val iterator0 = store.valuesIterator(dataToKeyRow("a", 0)) - assert(iterator0.hasNext) - assert(valueRowToData(iterator0.next()) === 1) - assert(!iterator0.hasNext) - - merge(store, "a", 0, 2) - merge(store, "a", 0, 3) - - val iterator1 = store.valuesIterator(dataToKeyRow("a", 0)) - - (1 to 3).map { i => - assert(iterator1.hasNext) - assert(valueRowToData(iterator1.next()) === i) + val cfName = if (colFamiliesEnabled) "testColFamily" else "default" + if (colFamiliesEnabled) { + store.createColFamilyIfAbsent(cfName, + keySchema, valueSchema, keyEncoder) } - assert(!iterator1.hasNext) - - remove(store, _._1 == "a") - val iterator2 = store.valuesIterator(dataToKeyRow("a", 0)) - assert(!iterator2.hasNext) + var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, + -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) + // put & get, iterator + timerTimestamps.foreach { ts => + val keyRow = if (ts < 0) { + dataToKeyRow("a", ts.toInt) + } else dataToKeyRow(ts.toString, ts.toInt) + val valueRow = dataToValueRow(1) + store.put(keyRow, valueRow, cfName) + assert(valueRowToData(store.get(keyRow, cfName)) === 1) + } + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + // remove + store.remove(dataToKeyRow(1L.toString, 1.toInt), cfName) + timerTimestamps = timerTimestamps.filter(_ != 1L) + assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) + + // prefix scan + if (!keyEncoder.getClass.toString.contains("No")) { + val keyRow = dataToPrefixKeyRow("a") + assert(store.prefixScan(keyRow, cfName).toSeq.length + == timerTimestamps.filter(_ < 0).length) + } - assert(get(store, "a", 0).isEmpty) + store.commit() } - } } } - Seq(true, false).foreach { virtualColFamilyEnabled => - test(s"validate rocksdb removeColFamilyIfExists correctness " + - s"with virtualColFamilyEnabled=$virtualColFamilyEnabled") { - Seq( - NoPrefixKeyStateEncoderSpec(keySchema), - PrefixKeyScanStateEncoderSpec(keySchema, 1), - RangeKeyScanStateEncoderSpec(keySchema, Seq(1)) - ).foreach { keyEncoder => + test(s"validate rocksdb removeColFamilyIfExists correctness") { + Seq( + NoPrefixKeyStateEncoderSpec(keySchema), + PrefixKeyScanStateEncoderSpec(keySchema, 1), + RangeKeyScanStateEncoderSpec(keySchema, Seq(1)) + ).foreach { keyEncoder => tryWithProviderResource(newStoreProvider(keySchema, keyEncoder, true)) { provider => val store = provider.getStore(0) val cfName = "testColFamily" store.createColFamilyIfAbsent(cfName, - keySchema, valueSchema, keyEncoder, - useVirtualColFamily = virtualColFamilyEnabled) + keySchema, valueSchema, keyEncoder) // put some test data into state store val timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, @@ -1028,6 +991,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid assert(store.removeColFamilyIfExists(cfName)) // TODO eliminate behavior difference + assert(!store.iterator(cfName).hasNext) + /* if (virtualColFamilyEnabled) { assert(!store.iterator(cfName).hasNext) } else { @@ -1041,8 +1006,8 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid sqlState = Some("42802"), parameters = Map("operationType" -> "iterator", "colFamilyName" -> cfName) ) - } - }} + } */ + } } } @@ -1067,13 +1032,11 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } def newStoreProvider(useColumnFamilies: Boolean, - useMultipleValuesPerKey: Boolean, - useVirtualColFamily: Boolean): RocksDBStateStoreProvider = { + useMultipleValuesPerKey: Boolean): RocksDBStateStoreProvider = { newStoreProvider(StateStoreId(newDir(), Random.nextInt(), 0), NoPrefixKeyStateEncoderSpec(keySchema), useColumnFamilies = useColumnFamilies, - useMultipleValuesPerKey = useMultipleValuesPerKey, - useVirtualColFamily = useVirtualColFamily + useMultipleValuesPerKey = useMultipleValuesPerKey ) } @@ -1098,8 +1061,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid sqlConf: Option[SQLConf] = None, conf: Configuration = new Configuration, useColumnFamilies: Boolean = false, - useMultipleValuesPerKey: Boolean = false, - useVirtualColFamily: Boolean = false): RocksDBStateStoreProvider = { + useMultipleValuesPerKey: Boolean = false): RocksDBStateStoreProvider = { val provider = new RocksDBStateStoreProvider() provider.init( storeId, 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 981569596d77b..6086fd43846f6 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 @@ -127,48 +127,34 @@ trait AlsoTestWithChangelogCheckpointingEnabled def testWithColumnFamilies( testName: String, testMode: TestMode, - alsoTestWithVirtualColFamily: Boolean, testTags: Tag*) - (testBody: (Boolean, Boolean) => Any): Unit = { + (testBody: Boolean => Any): Unit = { Seq(true, false).foreach { colFamiliesEnabled => - val virtualColFamilySeq = - if (alsoTestWithVirtualColFamily) Seq(true, false) else Seq(false) - - virtualColFamilySeq.foreach { virtualColFamilyEnabled => - val testNameSuffix = - if (alsoTestWithVirtualColFamily) { - s" & virtualColFamilyEnabled=$virtualColFamilyEnabled" - } else "" - testMode match { - case TestWithChangelogCheckpointingEnabled => - testWithChangelogCheckpointingEnabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, - testTags: _*) { - testBody(colFamiliesEnabled, virtualColFamilyEnabled) - } + testMode match { + case TestWithChangelogCheckpointingEnabled => + testWithChangelogCheckpointingEnabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { + testBody(colFamiliesEnabled) + } - case TestWithChangelogCheckpointingDisabled => - testWithChangelogCheckpointingDisabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, - testTags: _*) { - testBody(colFamiliesEnabled, virtualColFamilyEnabled) - } + case TestWithChangelogCheckpointingDisabled => + testWithChangelogCheckpointingDisabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { + testBody(colFamiliesEnabled) + } - case TestWithBothChangelogCheckpointingEnabledAndDisabled => - testWithChangelogCheckpointingEnabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, - testTags: _*) { - testBody(colFamiliesEnabled, virtualColFamilyEnabled) - } - testWithChangelogCheckpointingDisabled(s"$testName - " + - s"with colFamiliesEnabled=$colFamiliesEnabled" + testNameSuffix, - testTags: _*) { - testBody(colFamiliesEnabled, virtualColFamilyEnabled) - } + case TestWithBothChangelogCheckpointingEnabledAndDisabled => + testWithChangelogCheckpointingEnabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { + testBody(colFamiliesEnabled) + } + testWithChangelogCheckpointingDisabled(s"$testName - " + + s"with colFamiliesEnabled=$colFamiliesEnabled", testTags: _*) { + testBody(colFamiliesEnabled) + } - case _ => - throw new IllegalArgumentException(s"Unknown test mode: $testMode") - } + case _ => + throw new IllegalArgumentException(s"Unknown test mode: $testMode") } } } @@ -181,7 +167,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies( "RocksDB: check changelog and snapshot version", - TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => + TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 1) new File(remoteDir).delete() // to make sure that the directory gets created @@ -204,7 +190,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: load version that doesn't exist", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { + colFamiliesEnabled => val provider = new RocksDBStateStoreProvider() var ex = intercept[SparkException] { provider.getStore(-1) @@ -241,7 +228,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies( "RocksDB: purge changelog and snapshots", - TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => + TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val conf = dbConf.copy(enableChangelogCheckpointing = true, @@ -283,7 +270,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies( "RocksDB: minDeltasForSnapshot", - TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => + TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val conf = dbConf.copy(enableChangelogCheckpointing = true, minDeltasForSnapshot = 3) @@ -325,7 +312,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("SPARK-45419: Do not reuse SST files" + " in different RocksDB instances", - TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => + TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString val conf = dbConf.copy(minDeltasForSnapshot = 0, compactOnCommit = false) new File(remoteDir).delete() // to make sure that the directory gets created @@ -357,7 +344,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // an existing checkpoint without changelog. testWithColumnFamilies( "RocksDB: changelog checkpointing backward compatibility", - TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => + TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val disableChangelogCheckpointingConf = @@ -420,7 +407,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared // an existing checkpoint with changelog. testWithColumnFamilies( "RocksDB: changelog checkpointing forward compatibility", - TestWithChangelogCheckpointingEnabled, false) { (colFamiliesEnabled, _) => + TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val enableChangelogCheckpointingConf = @@ -472,7 +459,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: compression conf", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -488,7 +475,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: get, put, iterator, commit, load", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => def testOps(compactOnCommit: Boolean): Unit = { val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -558,7 +545,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: column family creation with invalid names", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -595,7 +582,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: column family creation with reserved chars", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -664,7 +651,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: operations on absent column family", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created @@ -700,7 +687,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies(s"RocksDB: get, put, iterator, commit, load " + s"with multiple column families", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val remoteDir = Utils.createTempDir().toString new File(remoteDir).delete() // to make sure that the directory gets created val colFamily1: String = "abc" @@ -816,7 +803,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies(s"RocksDB: handle commit failures and aborts", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val hadoopConf = new Configuration() hadoopConf.set( SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS.parent.key, @@ -1003,7 +990,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("RocksDBFileManager: create init dfs directory with " + s"unknown number of keys", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") try { val verificationDir = Utils.createTempDir().getAbsolutePath @@ -1093,7 +1080,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("RocksDBFileManager: delete orphan files", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath // Use 2 file managers here to emulate concurrent execution @@ -1173,7 +1160,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("RocksDBFileManager: don't delete orphan files " + s"when there is only 1 version", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath val fileManager = new RocksDBFileManager( @@ -1228,7 +1215,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("RocksDBFileManager: upload only new immutable files", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val dfsRootDir = dir.getAbsolutePath val verificationDir = Utils.createTempDir().getAbsolutePath // local dir to load checkpoints @@ -1318,7 +1305,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared testWithColumnFamilies("RocksDBFileManager: error writing [version].zip " + s"cancels the output stream", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => quietly { val hadoopConf = new Configuration() hadoopConf.set( @@ -1336,7 +1323,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("disallow concurrent updates to the same RocksDB instance", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => quietly { withDB( Utils.createTempDir().toString, @@ -1406,7 +1393,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("ensure concurrent access lock is released after Spark task completes", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => RocksDBSuite.withSingletonDB { // Load a RocksDB instance, that is, get a lock inside a task and then fail quietly { @@ -1424,7 +1411,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("checkpoint metadata serde roundtrip", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => // expect read metadata error when metadata uses unsupported version withTempDir { dir => val file2 = new File(dir, "json") @@ -1472,7 +1459,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("SPARK-36236: reset RocksDB metrics whenever a new version is loaded", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => def verifyMetrics(putCount: Long, getCount: Long, iterCountPositive: Boolean = false, metrics: RocksDBMetrics): Unit = { assert(metrics.nativeOpsHistograms("put").count === putCount, "invalid put count") @@ -1571,7 +1558,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("-1", "100", "1000").foreach { maxOpenFiles => testWithColumnFamilies(s"SPARK-39781: adding valid max_open_files=$maxOpenFiles " + "config property for RocksDB state store instance should succeed", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val sqlConf = SQLConf.get.clone() sqlConf.setConfString("spark.sql.streaming.stateStore.rocksdb.maxOpenFiles", maxOpenFiles) @@ -1594,7 +1581,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("test", "true").foreach { maxOpenFiles => testWithColumnFamilies(s"SPARK-39781: adding invalid max_open_files=$maxOpenFiles config " + "property for RocksDB state store instance should fail", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val ex = intercept[IllegalArgumentException] { val sqlConf = SQLConf.get.clone() @@ -1622,7 +1609,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("16", "32", "64").foreach {writeBufferSizeMB => testWithColumnFamilies(s"SPARK-42819: configure memtable memory usage with " + s"maxWriteBufferNumber=$maxWriteBufferNumber and writeBufferSize=$writeBufferSizeMB", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val sqlConf = new SQLConf sqlConf.setConfString("spark.sql.streaming.stateStore.rocksdb.maxWriteBufferNumber", @@ -1647,7 +1634,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("Verify that fallocate is allowed by default", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (_, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val sqlConf = new SQLConf val dbConf = RocksDBConf(StateStoreConf(sqlConf)) assert(dbConf.allowFAllocate == true) @@ -1655,7 +1642,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared /** RocksDB memory management tests for bounded memory usage */ testWithColumnFamilies("Memory mgmt - invalid config", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => try { RocksDBMemoryManager.resetWriteBufferManagerAndCache @@ -1694,8 +1681,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("true", "false").foreach { boundedMemoryUsage => testWithColumnFamilies(s"Memory mgmt - Cache reuse for RocksDB " + s"with boundedMemoryUsage=$boundedMemoryUsage", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { - (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir1 => withTempDir { dir2 => try { @@ -1755,7 +1741,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared Seq("100", "1000", "100000").foreach { totalMemorySizeMB => testWithColumnFamilies(s"Memory mgmt - valid config " + s"with totalMemorySizeMB=$totalMemorySizeMB", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => try { val sqlConf = new SQLConf @@ -1792,7 +1778,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } testWithColumnFamilies("SPARK-37224: flipping option 'trackTotalNumberOfRows' during restart", - TestWithBothChangelogCheckpointingEnabledAndDisabled, false) { (colFamiliesEnabled, _) => + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withTempDir { dir => val remoteDir = dir.getCanonicalPath From 167f8b3287f0baea29fe3f37a862c28768e9063b Mon Sep 17 00:00:00 2001 From: jingz-db Date: Tue, 2 Jul 2024 11:23:50 -0700 Subject: [PATCH 11/23] remove unused rocksdb code --- .../execution/streaming/state/RocksDB.scala | 88 ++++++------------- .../state/RocksDBStateStoreProvider.scala | 77 ++++++++++------ .../state/RocksDBStateStoreSuite.scala | 26 +++--- 3 files changed, 90 insertions(+), 101 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 088242b4246e5..c0fd0e44e2aa9 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 @@ -23,8 +23,6 @@ import java.util.concurrent.TimeUnit import javax.annotation.concurrent.GuardedBy import scala.collection.{mutable, Map} -import scala.collection.mutable.ArrayBuffer -import scala.jdk.CollectionConverters._ import scala.ref.WeakReference import scala.util.Try @@ -136,10 +134,10 @@ class RocksDB( dbOptions.setWriteBufferManager(writeBufferManager) } - // Maintain mapping of column family name to handle + // Maintain a set of column family name @GuardedBy("acquireLock") - private val colFamilyNameToHandleMap = - scala.collection.mutable.Map[String, ColumnFamilyHandle]() + private val colFamilyNameSet = + scala.collection.mutable.Set[String]() private val dbLogger = createLogger() // for forwarding RocksDB native logs to log4j dbOptions.setStatistics(new Statistics()) @@ -266,7 +264,7 @@ class RocksDB( * @return - true if the column family exists, false otherwise */ private def checkColFamilyExists(colFamilyName: String): Boolean = { - colFamilyNameToHandleMap.contains(colFamilyName) + colFamilyNameSet.contains(colFamilyName) } private val multColFamiliesDisabledStr = "multiple column families disabled in " + @@ -344,9 +342,7 @@ class RocksDB( verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) if (!checkColFamilyExists(colFamilyName)) { assert(db != null) - val descriptor = new ColumnFamilyDescriptor(colFamilyName.getBytes, columnFamilyOptions) - val handle = db.createColumnFamily(descriptor) - colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle + colFamilyNameSet += colFamilyName } } @@ -357,9 +353,7 @@ class RocksDB( verifyColFamilyCreationOrDeletion("remove_col_family", colFamilyName) if (checkColFamilyExists(colFamilyName)) { assert(db != null) - val handle = colFamilyNameToHandleMap(colFamilyName) - db.dropColumnFamily(handle) - colFamilyNameToHandleMap.remove(colFamilyName) + colFamilyNameSet.remove(colFamilyName) true } else { false @@ -374,7 +368,7 @@ class RocksDB( key: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = { verifyColFamilyOperations("get", colFamilyName) - db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) + db.get(readOptions, key) } /** @@ -387,13 +381,13 @@ class RocksDB( colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { verifyColFamilyOperations("put", colFamilyName) if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) + val oldValue = db.get(readOptions, key) if (oldValue == null) { numKeysOnWritingVersion += 1 } } - db.put(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value) + db.put(writeOptions, key, value) if (useColumnFamilies) { changelogWriter.foreach(_.put(key, value, colFamilyName)) } else { @@ -423,12 +417,12 @@ class RocksDB( verifyColFamilyOperations("merge", colFamilyName) if (conf.trackTotalNumberOfRows) { - val oldValue = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) + val oldValue = db.get(readOptions, key) if (oldValue == null) { numKeysOnWritingVersion += 1 } } - db.merge(colFamilyNameToHandleMap(colFamilyName), writeOptions, key, value) + db.merge(writeOptions, key, value) changelogWriter.foreach(_.merge(key, value, colFamilyName)) } @@ -442,12 +436,12 @@ class RocksDB( colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { verifyColFamilyOperations("remove", colFamilyName) if (conf.trackTotalNumberOfRows) { - val value = db.get(colFamilyNameToHandleMap(colFamilyName), readOptions, key) + val value = db.get(readOptions, key) if (value != null) { numKeysOnWritingVersion -= 1 } } - db.delete(colFamilyNameToHandleMap(colFamilyName), writeOptions, key) + db.delete(writeOptions, key) if (useColumnFamilies) { changelogWriter.foreach(_.delete(key, colFamilyName)) } else { @@ -462,7 +456,7 @@ class RocksDB( Iterator[ByteArrayPair] = { verifyColFamilyOperations("iterator", colFamilyName) - val iter = db.newIterator(colFamilyNameToHandleMap(colFamilyName)) + val iter = db.newIterator() logInfo(log"Getting iterator from version ${MDC(LogKeys.LOADED_VERSION, loadedVersion)}") iter.seekToFirst() @@ -490,7 +484,7 @@ class RocksDB( private def countKeys(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Long = { verifyColFamilyOperations("countKeys", colFamilyName) - val iter = db.newIterator(colFamilyNameToHandleMap(colFamilyName)) + val iter = db.newIterator() try { logInfo(log"Counting keys - getting iterator from version " + @@ -513,7 +507,7 @@ class RocksDB( def prefixScan(prefix: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Iterator[ByteArrayPair] = { verifyColFamilyOperations("prefixScan", colFamilyName) - val iter = db.newIterator(colFamilyNameToHandleMap(colFamilyName)) + val iter = db.newIterator() iter.seek(prefix) // Attempt to close this iterator if there is a task failure, or a task interruption. @@ -558,17 +552,16 @@ class RocksDB( // because rocksdb wal is disabled. logInfo(log"Flushing updates for ${MDC(LogKeys.VERSION_NUM, newVersion)}") flushTimeMs = timeTakenMs { - // Flush updates to all available column families - assert(!colFamilyNameToHandleMap.isEmpty) - db.flush(flushOptions, colFamilyNameToHandleMap.values.toSeq.asJava) + assert(!colFamilyNameSet.isEmpty) + db.flush(flushOptions) } if (conf.compactOnCommit) { logInfo("Compacting") compactTimeMs = timeTakenMs { // Perform compaction on all available column families - assert(!colFamilyNameToHandleMap.isEmpty) - colFamilyNameToHandleMap.values.foreach(db.compactRange(_)) + assert(!colFamilyNameSet.isEmpty) + db.compactRange() } } @@ -773,9 +766,9 @@ class RocksDB( } // Used for metrics reporting around internal/external column families - val numInternalColFamilies = colFamilyNameToHandleMap - .keys.filter(checkInternalColumnFamilies(_)).size - val numExternalColFamilies = colFamilyNameToHandleMap.keys.size - numInternalColFamilies + val numInternalColFamilies = colFamilyNameSet + .filter(checkInternalColumnFamilies(_)).size + val numExternalColFamilies = colFamilyNameSet.size - numInternalColFamilies // if bounded memory usage is enabled, we share the block cache across all state providers // running on the same node and account the usage to this single cache. In this case, its not @@ -873,45 +866,22 @@ class RocksDB( private def getDBProperty(property: String): Long = { // get cumulative sum across all available column families - assert(!colFamilyNameToHandleMap.isEmpty) - colFamilyNameToHandleMap - .values - .map(handle => db.getProperty(handle, property).toLong) - .sum + assert(!colFamilyNameSet.isEmpty) + db.getProperty(property).toLong } private def openDB(): Unit = { assert(db == null) - val colFamilies = NativeRocksDB.listColumnFamilies(dbOptions, workingDir.toString) + db = NativeRocksDB.open(dbOptions, workingDir.toString) - val colFamilyDescriptors = new ArrayBuffer[ColumnFamilyDescriptor] - // populate the list of available col family descriptors - colFamilies.asScala.toList.foreach { family => - val descriptor = new ColumnFamilyDescriptor(family, columnFamilyOptions) - colFamilyDescriptors += descriptor - } - - if (colFamilyDescriptors.isEmpty) { - colFamilyDescriptors += new ColumnFamilyDescriptor(NativeRocksDB.DEFAULT_COLUMN_FAMILY, - columnFamilyOptions) - } - - val colFamilyHandles = new java.util.ArrayList[ColumnFamilyHandle]() - db = NativeRocksDB.open(new DBOptions(dbOptions), workingDir.toString, - colFamilyDescriptors.asJava, colFamilyHandles) - - // Store the mapping of names to handles in the internal map - colFamilyHandles.asScala.toList.foreach { handle => - colFamilyNameToHandleMap(handle.getName.map(_.toChar).mkString) = handle - } + // TODO is the colFamilyNameSet still working? + colFamilyNameSet += StateStore.DEFAULT_COL_FAMILY_NAME logInfo(log"Opened DB with conf ${MDC(LogKeys.CONFIG, conf)}") } private def closeDB(): Unit = { if (db != null) { - // Close the column family handles in case multiple column families are used - colFamilyNameToHandleMap.values.map(handle => handle.close) - colFamilyNameToHandleMap.clear() + colFamilyNameSet.clear() // Cancel and wait until all background work finishes db.cancelAllBackgroundWork(true) 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 d88ca08bff8d2..12de7e303ed17 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 @@ -60,6 +60,7 @@ private[sql] class RocksDBStateStoreProvider verify(colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME, s"Failed to create column family with reserved_name=$colFamilyName") verify(useColumnFamilies, "Column families are not supported in this store") + rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) // TODO how to efficiently guarantee there isn't any value conflict for different key def getNextRandShort: Short = { @@ -72,13 +73,22 @@ private[sql] class RocksDBStateStoreProvider RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) } + private def getEncodedKey( + keyEncoder: RocksDBKeyStateEncoder, + key: UnsafeRow, + colFamilyName: String): Array[Byte] = { + if (useColumnFamilies) { + keyEncoder.encodeKey(key, colFamilyToIdMap.get(colFamilyName)) + } else keyEncoder.encodeKey(key) + } + override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val value = kvEncoder._2.decodeValue( - rocksDB.get(kvEncoder._1.encodeKey(key, - colFamilyToIdMap.get(colFamilyName)))) + val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) + val value = + kvEncoder._2.decodeValue(rocksDB.get(encodedKey, colFamilyName)) if (!isValidated && value != null && !useColumnFamilies) { StateStoreProvider.validateStateRowFormat( @@ -108,8 +118,8 @@ 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, colFamilyToIdMap.get(colFamilyName))) + val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) + val encodedValues = rocksDB.get(encodedKey, colFamilyName) valueEncoder.decodeValues(encodedValues) } @@ -123,11 +133,8 @@ private[sql] class RocksDBStateStoreProvider " which supports multiple values for a single key") verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - val (encodedKey, physicalColFamilyName) = - (keyEncoder.encodeKey(key, colFamilyToIdMap.get(colFamilyName)), - StateStore.DEFAULT_COL_FAMILY_NAME) - - rocksDB.merge(encodedKey, valueEncoder.encodeValue(value), physicalColFamilyName) + val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) + rocksDB.merge(encodedKey, valueEncoder.encodeValue(value), colFamilyName) } override def put(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = { @@ -135,31 +142,45 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot put a null value") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - rocksDB.put(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName)), - kvEncoder._2.encodeValue(value)) + val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) + rocksDB.put(encodedKey, kvEncoder._2.encodeValue(value), colFamilyName) } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { verify(state == UPDATING, "Cannot remove after already committed or aborted") verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - rocksDB.remove(kvEncoder._1.encodeKey(key, colFamilyToIdMap.get(colFamilyName))) + val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) + rocksDB.remove(encodedKey, colFamilyName) } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() - val cfId: Short = colFamilyToIdMap.get(colFamilyName) - rocksDB.prefixScan(getIdBytes(cfId)).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), - kvEncoder._2.decodeValue(kv.value)) - if (!isValidated && rowPair.value != null && !useColumnFamilies) { - StateStoreProvider.validateStateRowFormat( - rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) - isValidated = true + if (useColumnFamilies) { + val cfId: Short = colFamilyToIdMap.get(colFamilyName) + rocksDB.prefixScan(getIdBytes(cfId), colFamilyName).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), + kvEncoder._2.decodeValue(kv.value)) + if (!isValidated && rowPair.value != null && !useColumnFamilies) { + StateStoreProvider.validateStateRowFormat( + rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) + isValidated = true + } + rowPair + } + } else { + rocksDB.iterator().map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key), + kvEncoder._2.decodeValue(kv.value)) + if (!isValidated && rowPair.value != null && !useColumnFamilies) { + StateStoreProvider.validateStateRowFormat( + rowPair.key, keySchema, rowPair.value, valueSchema, storeConf) + isValidated = true + } + rowPair } - rowPair } } @@ -170,10 +191,12 @@ private[sql] class RocksDBStateStoreProvider "Prefix scan requires setting prefix key!") val rowPair = new UnsafeRowPair() + val prefix = if (useColumnFamilies) { + kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToIdMap.get(colFamilyName)) + } else kvEncoder._1.encodePrefixKey(prefixKey) - val prefix = kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToIdMap.get(colFamilyName)) - rocksDB.prefixScan(prefix).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), + rocksDB.prefixScan(prefix, colFamilyName).map { kv => + rowPair.withRows(kvEncoder._1.decodeKey(kv.key, useColumnFamilies), kvEncoder._2.decodeValue(kv.value)) rowPair } @@ -292,13 +315,15 @@ private[sql] class RocksDBStateStoreProvider colFamilyToIdMap.get(colFamilyName) ) var colFamilyExists = false - rocksDB.prefixScan(idPrefix).foreach { kv => + rocksDB.prefixScan(idPrefix, colFamilyName).foreach { kv => colFamilyExists = true rocksDB.remove(kv.key) } colFamilyExists } + rocksDB.removeColFamilyIfExists(colFamilyName) colFamilyToIdMap.remove(colFamilyName) + keyValueEncoderMap.remove(colFamilyName) result } } 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 fa4cd306a512a..1c7343badcaeb 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 @@ -990,23 +990,17 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid // assert col family existence assert(store.removeColFamilyIfExists(cfName)) - // TODO eliminate behavior difference - assert(!store.iterator(cfName).hasNext) - /* - if (virtualColFamilyEnabled) { - assert(!store.iterator(cfName).hasNext) - } else { - val e = intercept[Exception] { - store.iterator(cfName) - } + val e = intercept[Exception] { + store.iterator(cfName) + } - checkError( - exception = e.asInstanceOf[StateStoreUnsupportedOperationOnMissingColumnFamily], - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", - sqlState = Some("42802"), - parameters = Map("operationType" -> "iterator", "colFamilyName" -> cfName) - ) - } */ + checkError( + exception = e.asInstanceOf[StateStoreUnsupportedOperationOnMissingColumnFamily], + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + sqlState = Some("42802"), + // TODO how to throw error with iterator? + parameters = Map("operationType" -> "prefixScan", "colFamilyName" -> cfName) + ) } } } From 1a48930664bc015a18580f1635c7820f110a5906 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 11:19:15 -0700 Subject: [PATCH 12/23] RocksDBSuite to be fixed --- .../StatefulProcessorHandleImpl.scala | 1 + .../execution/streaming/state/RocksDB.scala | 186 ++------------- .../streaming/state/RocksDBStateEncoder.scala | 149 ++++-------- .../state/RocksDBStateStoreProvider.scala | 219 ++++++++++++++---- .../streaming/state/StateStoreChangelog.scala | 63 ++--- .../state/RocksDBStateStoreSuite.scala | 6 +- .../streaming/state/RocksDBSuite.scala | 32 +-- .../streaming/TransformWithStateSuite.scala | 6 +- 8 files changed, 272 insertions(+), 390 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala index dcc77e94de280..90d823d17dd7b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala @@ -130,6 +130,7 @@ class StatefulProcessorHandleImpl( valEncoder: Encoder[T]): ValueState[T] = { verifyStateVarOperations("get_value_state") incrementMetric("numValueStateVars") + println("I am inside getValueState, after inc metric: " + metrics.get("numValueStateVars")) val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder) resultState } 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 c0fd0e44e2aa9..d2c8e6cbc2036 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 @@ -62,7 +62,6 @@ case object StoreMaintenance extends RocksDBOpType("store_maintenance") * @param localRootDir Root directory in local disk that is used to working and checkpointing dirs * @param hadoopConf Hadoop configuration for talking to the remote file system * @param loggingId Id that will be prepended in logs for isolating concurrent RocksDBs - * @param useColumnFamilies Used to determine whether a single or multiple column families are used */ class RocksDB( dfsRootDir: String, @@ -134,11 +133,6 @@ class RocksDB( dbOptions.setWriteBufferManager(writeBufferManager) } - // Maintain a set of column family name - @GuardedBy("acquireLock") - private val colFamilyNameSet = - scala.collection.mutable.Set[String]() - private val dbLogger = createLogger() // for forwarding RocksDB native logs to log4j dbOptions.setStatistics(new Statistics()) private val nativeStats = dbOptions.statistics() @@ -235,20 +229,16 @@ class RocksDB( var changelogReader: StateStoreChangelogReader = null try { changelogReader = fileManager.getChangelogReader(v, useColumnFamilies) - changelogReader.foreach { case (recordType, key, value, colFamilyName) => - if (useColumnFamilies && !checkColFamilyExists(colFamilyName)) { - createColFamilyIfAbsent(colFamilyName, checkInternalColumnFamilies(colFamilyName)) - } - + changelogReader.foreach { case (recordType, key, value) => recordType match { case RecordType.PUT_RECORD => - put(key, value, colFamilyName) + put(key, value) case RecordType.DELETE_RECORD => - remove(key, colFamilyName) + remove(key) case RecordType.MERGE_RECORD => - merge(key, value, colFamilyName) + merge(key, value) } } } finally { @@ -258,116 +248,11 @@ class RocksDB( loadedVersion = endVersion } - /** - * Function to check if the column family exists in the state store instance. - * @param colFamilyName - name of the column family - * @return - true if the column family exists, false otherwise - */ - private def checkColFamilyExists(colFamilyName: String): Boolean = { - colFamilyNameSet.contains(colFamilyName) - } - - private val multColFamiliesDisabledStr = "multiple column families disabled in " + - "RocksDBStateStoreProvider" - - /** - * Function to verify invariants for column family based operations such as get, put, remove etc. - * @param operationName - name of the store operation - * @param colFamilyName - name of the column family - */ - private def verifyColFamilyOperations( - operationName: String, - colFamilyName: String): Unit = { - if (colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME) { - // if the state store instance does not support multiple column families, throw an exception - if (!useColumnFamilies) { - throw StateStoreErrors.unsupportedOperationException(operationName, - multColFamiliesDisabledStr) - } - - // if the column family name is empty or contains leading/trailing whitespaces, throw an - // exception - if (colFamilyName.isEmpty || colFamilyName.trim != colFamilyName) { - throw StateStoreErrors.cannotUseColumnFamilyWithInvalidName(operationName, colFamilyName) - } - - // if the column family does not exist, throw an exception - if (!checkColFamilyExists(colFamilyName)) { - throw StateStoreErrors.unsupportedOperationOnMissingColumnFamily(operationName, - colFamilyName) - } - } - } - - /** - * Function to verify invariants for column family creation or deletion operations. - * @param operationName - name of the store operation - * @param colFamilyName - name of the column family - */ - private def verifyColFamilyCreationOrDeletion( - operationName: String, - colFamilyName: String, - isInternal: Boolean = false): Unit = { - // if the state store instance does not support multiple column families, throw an exception - if (!useColumnFamilies) { - throw StateStoreErrors.unsupportedOperationException(operationName, - multColFamiliesDisabledStr) - } - - // if the column family name is empty or contains leading/trailing whitespaces - // or using the reserved "default" column family, throw an exception - if (colFamilyName.isEmpty - || colFamilyName.trim != colFamilyName - || colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) { - throw StateStoreErrors.cannotUseColumnFamilyWithInvalidName(operationName, colFamilyName) - } - - // if the column family is not internal and uses reserved characters, throw an exception - if (!isInternal && colFamilyName.charAt(0) == '_') { - throw StateStoreErrors.cannotCreateColumnFamilyWithReservedChars(colFamilyName) - } - } - - /** - * 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) == '_' - - /** - * Create RocksDB column family, if not created already - */ - def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean = false): Unit = { - verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) - if (!checkColFamilyExists(colFamilyName)) { - assert(db != null) - colFamilyNameSet += colFamilyName - } - } - - /** - * Remove RocksDB column family, if exists - */ - def removeColFamilyIfExists(colFamilyName: String): Boolean = { - verifyColFamilyCreationOrDeletion("remove_col_family", colFamilyName) - if (checkColFamilyExists(colFamilyName)) { - assert(db != null) - colFamilyNameSet.remove(colFamilyName) - true - } else { - false - } - } - /** * 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], - colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Array[Byte] = { - verifyColFamilyOperations("get", colFamilyName) + def get(key: Array[Byte]): Array[Byte] = { db.get(readOptions, key) } @@ -375,11 +260,7 @@ 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], - colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - verifyColFamilyOperations("put", colFamilyName) + def put(key: Array[Byte], value: Array[Byte]): Unit = { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, key) if (oldValue == null) { @@ -388,11 +269,7 @@ class RocksDB( } db.put(writeOptions, key, value) - if (useColumnFamilies) { - changelogWriter.foreach(_.put(key, value, colFamilyName)) - } else { - changelogWriter.foreach(_.put(key, value)) - } + changelogWriter.foreach(_.put(key, value)) } /** @@ -406,15 +283,7 @@ class RocksDB( * * @note This update is not committed to disk until commit() is called. */ - def merge( - key: Array[Byte], - value: Array[Byte], - colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - if (!useColumnFamilies) { - throw StateStoreErrors.unsupportedOperationException("merge", - multColFamiliesDisabledStr) - } - verifyColFamilyOperations("merge", colFamilyName) + def merge(key: Array[Byte], value: Array[Byte]): Unit = { if (conf.trackTotalNumberOfRows) { val oldValue = db.get(readOptions, key) @@ -424,17 +293,14 @@ class RocksDB( } db.merge(writeOptions, key, value) - changelogWriter.foreach(_.merge(key, value, colFamilyName)) + changelogWriter.foreach(_.merge(key, value)) } /** * Remove the key if present. * @note This update is not committed to disk until commit() is called. */ - def remove( - key: Array[Byte], - colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { - verifyColFamilyOperations("remove", colFamilyName) + def remove(key: Array[Byte]): Unit = { if (conf.trackTotalNumberOfRows) { val value = db.get(readOptions, key) if (value != null) { @@ -442,19 +308,14 @@ class RocksDB( } } db.delete(writeOptions, key) - if (useColumnFamilies) { - changelogWriter.foreach(_.delete(key, colFamilyName)) - } else { - changelogWriter.foreach(_.delete(key)) - } + changelogWriter.foreach(_.delete(key)) } /** * Get an iterator of all committed and uncommitted key-value pairs. */ - def iterator(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): + def iterator(): Iterator[ByteArrayPair] = { - verifyColFamilyOperations("iterator", colFamilyName) val iter = db.newIterator() logInfo(log"Getting iterator from version ${MDC(LogKeys.LOADED_VERSION, loadedVersion)}") @@ -482,8 +343,7 @@ class RocksDB( } } - private def countKeys(colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Long = { - verifyColFamilyOperations("countKeys", colFamilyName) + private def countKeys(): Long = { val iter = db.newIterator() try { @@ -504,9 +364,8 @@ class RocksDB( } } - def prefixScan(prefix: Array[Byte], colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): + def prefixScan(prefix: Array[Byte]): Iterator[ByteArrayPair] = { - verifyColFamilyOperations("prefixScan", colFamilyName) val iter = db.newIterator() iter.seek(prefix) @@ -552,15 +411,12 @@ class RocksDB( // because rocksdb wal is disabled. logInfo(log"Flushing updates for ${MDC(LogKeys.VERSION_NUM, newVersion)}") flushTimeMs = timeTakenMs { - assert(!colFamilyNameSet.isEmpty) db.flush(flushOptions) } if (conf.compactOnCommit) { logInfo("Compacting") compactTimeMs = timeTakenMs { - // Perform compaction on all available column families - assert(!colFamilyNameSet.isEmpty) db.compactRange() } } @@ -765,11 +621,6 @@ class RocksDB( nativeStats.getTickerCount(typ) } - // Used for metrics reporting around internal/external column families - val numInternalColFamilies = colFamilyNameSet - .filter(checkInternalColumnFamilies(_)).size - val numExternalColFamilies = colFamilyNameSet.size - numInternalColFamilies - // if bounded memory usage is enabled, we share the block cache across all state providers // running on the same node and account the usage to this single cache. In this case, its not // possible to provide partition level or query level memory usage. @@ -791,8 +642,6 @@ class RocksDB( filesCopied = fileManagerMetrics.filesCopied, filesReused = fileManagerMetrics.filesReused, zipFileBytesUncompressed = fileManagerMetrics.zipFileBytesUncompressed, - numExternalColFamilies = numExternalColFamilies, - numInternalColFamilies = numInternalColFamilies, nativeOpsMetrics = nativeOpsMetrics) } @@ -866,22 +715,17 @@ class RocksDB( private def getDBProperty(property: String): Long = { // get cumulative sum across all available column families - assert(!colFamilyNameSet.isEmpty) db.getProperty(property).toLong } private def openDB(): Unit = { assert(db == null) db = NativeRocksDB.open(dbOptions, workingDir.toString) - - // TODO is the colFamilyNameSet still working? - colFamilyNameSet += StateStore.DEFAULT_COL_FAMILY_NAME logInfo(log"Opened DB with conf ${MDC(LogKeys.CONFIG, conf)}") } private def closeDB(): Unit = { if (db != null) { - colFamilyNameSet.clear() // Cancel and wait until all background work finishes db.cancelAllBackgroundWork(true) @@ -1180,8 +1024,6 @@ case class RocksDBMetrics( bytesCopied: Long, filesReused: Long, zipFileBytesUncompressed: Option[Long], - numExternalColFamilies: Long, - numInternalColFamilies: Long, nativeOpsMetrics: Map[String, Long]) { def json: String = Serialization.write(this)(RocksDBMetrics.format) } 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 c622424f317aa..4cd89c31f9f08 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 @@ -30,12 +30,9 @@ import org.apache.spark.unsafe.Platform sealed trait RocksDBKeyStateEncoder { def supportPrefixKeyScan: Boolean - def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] - // TODO try change less of the API for key encoder? - def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] - def encodeKey(row: UnsafeRow): Array[Byte] - def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] - def decodeKey(keyBytes: Array[Byte], hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow + def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] + def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] + def decodeKey(keyBytes: Array[Byte], hasVcfPrefix: Boolean = false): UnsafeRow } sealed trait RocksDBValueStateEncoder { @@ -150,10 +147,8 @@ class PrefixKeyScanStateEncoder( // Reusable objects private val joinedRowOnKey = new JoinedRow() - private def encodeKeyWithColFamilyPrefix( - row: UnsafeRow, - hasVirtualColFamilyPrefix: Boolean, - colFamilyId: Short = 0): Array[Byte] = { + override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + val hasVirtualColFamilyPrefix: Boolean = vcfId.isDefined val prefixKeyEncoded = encodeUnsafeRow(extractPrefixKey(row)) val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) val offSetForColFamilyPrefix = @@ -162,7 +157,7 @@ class PrefixKeyScanStateEncoder( val encodedBytes = new Array[Byte](prefixKeyEncoded.length + remainingEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -180,19 +175,11 @@ class PrefixKeyScanStateEncoder( encodedBytes } - override def encodeKey(row: UnsafeRow): Array[Byte] = { - encodeKeyWithColFamilyPrefix(row, false) - } - - override def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] = { - encodeKeyWithColFamilyPrefix(row, true, colFamilyId) - } - - private def decodeKeyWithColFamilyPrefix( + override def decodeKey( keyBytes: Array[Byte], - hasVirtualColFamilyPrefix: Boolean): UnsafeRow = { + hasVcfPrefix: Boolean = false): UnsafeRow = { val offSetForColFamilyPrefix = - if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + if (hasVcfPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val prefixKeyEncodedLen = Platform.getInt( keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) @@ -218,20 +205,13 @@ class PrefixKeyScanStateEncoder( restoreKeyProjection(joinedRowOnKey.withLeft(prefixKeyDecoded).withRight(remainingKeyDecoded)) } - override def decodeKey( - keyBytes: Array[Byte], - hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow = { - decodeKeyWithColFamilyPrefix(keyBytes, hasVirtualColFamilyPrefix) - } - private def extractPrefixKey(key: UnsafeRow): UnsafeRow = { prefixKeyProjection(key) } - private def encodePrefixKeyWithColFamilyPrefix( - prefixKey: UnsafeRow, - hasVirtualColFamilyPrefix: Boolean = false, - colFamilyId: Short = 0): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + val hasVirtualColFamilyPrefix = vcfId.isDefined + val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 @@ -239,7 +219,7 @@ class PrefixKeyScanStateEncoder( val prefix = new Array[Byte]( prefixKeyEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, prefixKeyEncoded.length) @@ -249,14 +229,6 @@ class PrefixKeyScanStateEncoder( prefix } - override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { - encodePrefixKeyWithColFamilyPrefix(prefixKey) - } - - override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] = { - encodePrefixKeyWithColFamilyPrefix(prefixKey, true, colFamilyId) - } - override def supportPrefixKeyScan: Boolean = true } @@ -550,10 +522,8 @@ class RangeKeyScanStateEncoder( writer.getRow() } - private def encodeKeyWithColFamilyPrefix( - row: UnsafeRow, - hasVirtualColFamilyPrefix: Boolean = false, - colFamilyId: Short = 0): Array[Byte] = { + override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + val hasVirtualColFamilyPrefix: Boolean = vcfId.isDefined // This prefix key has the columns specified by orderingOrdinals val prefixKey = extractPrefixKey(row) val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) @@ -567,7 +537,7 @@ class RangeKeyScanStateEncoder( remainingEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -588,7 +558,7 @@ class RangeKeyScanStateEncoder( val encodedBytes = new Array[Byte]( rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -601,19 +571,11 @@ class RangeKeyScanStateEncoder( result } - override def encodeKey(row: UnsafeRow): Array[Byte] = { - encodeKeyWithColFamilyPrefix(row, false) - } - - override def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] = { - encodeKeyWithColFamilyPrefix(row, true, colFamilyId) - } - - private def decodeKeyWithColFamilyPrefix( + override def decodeKey( keyBytes: Array[Byte], - hasVirtualColFamilyPrefix: Boolean): UnsafeRow = { + hasVcfPrefix: Boolean = false): UnsafeRow = { val offSetForColFamilyPrefix = - if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + if (hasVcfPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val prefixKeyEncodedLen = Platform.getInt( keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) @@ -633,7 +595,7 @@ class RangeKeyScanStateEncoder( val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix + - prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, + prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) val remainingKeyDecoded = decodeToUnsafeRow(remainingKeyEncoded, @@ -649,16 +611,8 @@ class RangeKeyScanStateEncoder( } } - override def decodeKey( - keyBytes: Array[Byte], - hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow = { - decodeKeyWithColFamilyPrefix(keyBytes, hasVirtualColFamilyPrefix) - } - - private def encodePrefixKeyWithColFamilyPrefix( - prefixKey: UnsafeRow, - hasVirtualColFamilyPrefix: Boolean = false, - colFamilyId: Short = 0): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + val hasVirtualColFamilyPrefix: Boolean = vcfId.isDefined val offSetForColFamilyPrefix = if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 @@ -666,7 +620,7 @@ class RangeKeyScanStateEncoder( val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) if (hasVirtualColFamilyPrefix) { - Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, colFamilyId) + Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, rangeScanKeyEncoded.length) @@ -675,14 +629,6 @@ class RangeKeyScanStateEncoder( prefix } - override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { - encodePrefixKeyWithColFamilyPrefix(prefixKey, false) - } - - override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] = { - encodePrefixKeyWithColFamilyPrefix(prefixKey, true, colFamilyId) - } - override def supportPrefixKeyScan: Boolean = true } @@ -706,23 +652,25 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) // Reusable objects private val keyRow = new UnsafeRow(keySchema.size) - override def encodeKey(row: UnsafeRow): Array[Byte] = encodeUnsafeRow(row) - - override def encodeKey(row: UnsafeRow, colFamilyId: Short): Array[Byte] = { - val bytesToEncode = row.getBytes - val encodedBytes = new Array[Byte](bytesToEncode.length + - STATE_ENCODING_NUM_VERSION_BYTES + VIRTUAL_COL_FAMILY_PREFIX_BYTES) - - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, colFamilyId) - Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES, - STATE_ENCODING_VERSION) - // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. - Platform.copyMemory( - bytesToEncode, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + - VIRTUAL_COL_FAMILY_PREFIX_BYTES, - bytesToEncode.length) - encodedBytes + override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + if (!vcfId.isDefined) { + encodeUnsafeRow(row) + } else { + val bytesToEncode = row.getBytes + val encodedBytes = new Array[Byte](bytesToEncode.length + + STATE_ENCODING_NUM_VERSION_BYTES + VIRTUAL_COL_FAMILY_PREFIX_BYTES) + + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) + Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES, + STATE_ENCODING_VERSION) + // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. + Platform.copyMemory( + bytesToEncode, Platform.BYTE_ARRAY_OFFSET, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + + VIRTUAL_COL_FAMILY_PREFIX_BYTES, + bytesToEncode.length) + encodedBytes + } } /** @@ -732,8 +680,8 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) */ override def decodeKey( keyBytes: Array[Byte], - hasVirtualColFamilyPrefix: Boolean = false): UnsafeRow = { - if (hasVirtualColFamilyPrefix) { + hasVcfPrefix: Boolean = false): UnsafeRow = { + if (hasVcfPrefix) { if (keyBytes != null) { // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. keyRow.pointTo( @@ -750,14 +698,9 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) override def supportPrefixKeyScan: Boolean = false - override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { throw new IllegalStateException("This encoder doesn't support prefix key!") } - - override def encodePrefixKey(prefixKey: UnsafeRow, colFamilyId: Short): Array[Byte] = { - throw new IllegalArgumentException("This encoder doesn't support prefix key encoding" + - "with column family Id!") - } } /** 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 12de7e303ed17..43a4ef107cb2f 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 @@ -18,7 +18,9 @@ package org.apache.spark.sql.execution.streaming.state import java.io._ +import java.util.concurrent.atomic.AtomicInteger +import scala.jdk.CollectionConverters._ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration @@ -60,35 +62,21 @@ private[sql] class RocksDBStateStoreProvider verify(colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME, s"Failed to create column family with reserved_name=$colFamilyName") verify(useColumnFamilies, "Column families are not supported in this store") - rocksDB.createColFamilyIfAbsent(colFamilyName, isInternal) - - // TODO how to efficiently guarantee there isn't any value conflict for different key - def getNextRandShort: Short = { - (scala.util.Random.nextInt(Short.MaxValue - Short.MinValue + 1) + Short.MinValue).toShort - } - colFamilyToIdMap.putIfAbsent(colFamilyName, getNextRandShort) + ColumnFamilyUtils.createColFamilyIfAbsent(colFamilyName, isInternal) keyValueEncoderMap.putIfAbsent(colFamilyName, (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec), RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) } - private def getEncodedKey( - keyEncoder: RocksDBKeyStateEncoder, - key: UnsafeRow, - colFamilyName: String): Array[Byte] = { - if (useColumnFamilies) { - keyEncoder.encodeKey(key, colFamilyToIdMap.get(colFamilyName)) - } else keyEncoder.encodeKey(key) - } - override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) + val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + ColumnFamilyUtils.verifyColFamilyOperations("get", colFamilyName) val value = - kvEncoder._2.decodeValue(rocksDB.get(encodedKey, colFamilyName)) + kvEncoder._2.decodeValue(rocksDB.get(encodedKey)) if (!isValidated && value != null && !useColumnFamilies) { StateStoreProvider.validateStateRowFormat( @@ -118,8 +106,9 @@ private[sql] class RocksDBStateStoreProvider verify(valueEncoder.supportsMultipleValuesPerKey, "valuesIterator requires a encoder " + "that supports multiple values for a single key.") - val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) - val encodedValues = rocksDB.get(encodedKey, colFamilyName) + val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + ColumnFamilyUtils.verifyColFamilyOperations("get", colFamilyName) + val encodedValues = rocksDB.get(encodedKey) valueEncoder.decodeValues(encodedValues) } @@ -133,8 +122,10 @@ private[sql] class RocksDBStateStoreProvider " which supports multiple values for a single key") verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) - rocksDB.merge(encodedKey, valueEncoder.encodeValue(value), colFamilyName) + val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + + ColumnFamilyUtils.verifyColFamilyOperations("merge", colFamilyName) + rocksDB.merge(encodedKey, valueEncoder.encodeValue(value)) } override def put(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = { @@ -142,16 +133,20 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot put a null value") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) - rocksDB.put(encodedKey, kvEncoder._2.encodeValue(value), colFamilyName) + val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + ColumnFamilyUtils.verifyColFamilyOperations("put", colFamilyName) + + rocksDB.put(encodedKey, kvEncoder._2.encodeValue(value)) } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { verify(state == UPDATING, "Cannot remove after already committed or aborted") verify(key != null, "Key cannot be null") val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = getEncodedKey(kvEncoder._1, key, colFamilyName) - rocksDB.remove(encodedKey, colFamilyName) + val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + + ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) + rocksDB.remove(encodedKey) } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { @@ -160,7 +155,10 @@ private[sql] class RocksDBStateStoreProvider if (useColumnFamilies) { val cfId: Short = colFamilyToIdMap.get(colFamilyName) - rocksDB.prefixScan(getIdBytes(cfId), colFamilyName).map { kv => + // Note this verify function only verify on the colFamilyName being valid, + // we are actually doing prefix, but pass "iterator" to throw correct error message + ColumnFamilyUtils.verifyColFamilyOperations("iterator", colFamilyName) + rocksDB.prefixScan(ColumnFamilyUtils.getVcfIdBytes(cfId)).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), kvEncoder._2.decodeValue(kv.value)) if (!isValidated && rowPair.value != null && !useColumnFamilies) { @@ -171,6 +169,7 @@ private[sql] class RocksDBStateStoreProvider rowPair } } else { + ColumnFamilyUtils.verifyColFamilyOperations("iterator", colFamilyName) rocksDB.iterator().map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) @@ -191,11 +190,11 @@ private[sql] class RocksDBStateStoreProvider "Prefix scan requires setting prefix key!") val rowPair = new UnsafeRowPair() - val prefix = if (useColumnFamilies) { - kvEncoder._1.encodePrefixKey(prefixKey, colFamilyToIdMap.get(colFamilyName)) - } else kvEncoder._1.encodePrefixKey(prefixKey) + val prefix = + kvEncoder._1.encodePrefixKey(prefixKey, Option(colFamilyToIdMap.get(colFamilyName))) - rocksDB.prefixScan(prefix, colFamilyName).map { kv => + ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) + rocksDB.prefixScan(prefix).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, useColumnFamilies), kvEncoder._2.decodeValue(kv.value)) rowPair @@ -249,6 +248,17 @@ private[sql] class RocksDBStateStoreProvider rocksDBMetrics.nativeOpsMetrics.getOrElse(typ, 0) } + // Used for metrics reporting around internal/external column families + def internalColFamilyCnt(): Long = { + colFamilyToIdMap.keys.asScala.toSeq + .filter(ColumnFamilyUtils.checkInternalColumnFamilies(_)).size + } + + def externalColFamilyCnt(): Long = { + colFamilyToIdMap.keys.asScala.toSeq + .filter(!ColumnFamilyUtils.checkInternalColumnFamilies(_)).size + } + val stateStoreCustomMetrics = Map[StateStoreCustomMetric, Long]( CUSTOM_METRIC_SST_FILE_SIZE -> rocksDBMetrics.totalSSTFilesBytes, CUSTOM_METRIC_GET_TIME -> sumNativeOpsLatencyMillis("get"), @@ -273,8 +283,8 @@ 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_EXTERNAL_COL_FAMILIES -> rocksDBMetrics.numExternalColFamilies, - CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES -> rocksDBMetrics.numInternalColFamilies + CUSTOM_METRIC_NUM_EXTERNAL_COL_FAMILIES -> internalColFamilyCnt(), + CUSTOM_METRIC_NUM_INTERNAL_COL_FAMILIES -> externalColFamilyCnt() ) ++ rocksDBMetrics.zipFileBytesUncompressed.map(bytes => Map(CUSTOM_METRIC_ZIP_FILE_BYTES_UNCOMPRESSED -> bytes)).getOrElse(Map()) @@ -299,30 +309,24 @@ private[sql] class RocksDBStateStoreProvider /** Return the [[RocksDB]] instance in this store. This is exposed mainly for testing. */ def dbInstance(): RocksDB = rocksDB - // TODO How to avoid memcpy here - private def getIdBytes(id: Short): Array[Byte] = { - val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) - encodedBytes - } - /** Remove column family if exists */ override def removeColFamilyIfExists(colFamilyName: String): Boolean = { verify(useColumnFamilies, "Column families are not supported in this store") val result = { - // TODO more efficient way to do remove col family? - val idPrefix = getIdBytes( + val idPrefix = ColumnFamilyUtils.getVcfIdBytes( colFamilyToIdMap.get(colFamilyName) ) - var colFamilyExists = false - rocksDB.prefixScan(idPrefix, colFamilyName).foreach { kv => - colFamilyExists = true - rocksDB.remove(kv.key) + val colFamilyExists = ColumnFamilyUtils.checkColFamilyExists(colFamilyName) + if (colFamilyExists) { + ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) + rocksDB.prefixScan(idPrefix).foreach { kv => + ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) + rocksDB.remove(kv.key) + } } colFamilyExists } - rocksDB.removeColFamilyIfExists(colFamilyName) - colFamilyToIdMap.remove(colFamilyName) + ColumnFamilyUtils.removeColFamilyIfExists(colFamilyName) keyValueEncoderMap.remove(colFamilyName) result } @@ -352,6 +356,10 @@ private[sql] class RocksDBStateStoreProvider keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec), RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) + if (useColumnFamilies) { + // put default column family only if useColumnFamilies are enabled + colFamilyToIdMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, colFamilyId.shortValue()) + } rocksDB // lazy initialization } @@ -427,10 +435,125 @@ private[sql] class RocksDBStateStoreProvider (RocksDBKeyStateEncoder, RocksDBValueStateEncoder)] private val colFamilyToIdMap = new java.util.concurrent.ConcurrentHashMap[String, Short] + // TODO SPARK-48796 load column family id from state schema when restarting + private val colFamilyId = new AtomicInteger(0) private def verify(condition: => Boolean, msg: String): Unit = { if (!condition) { throw new IllegalStateException(msg) } } + + private object ColumnFamilyUtils { + private val multColFamiliesDisabledStr = "multiple column families disabled in " + + "RocksDBStateStoreProvider" + + def getVcfIdBytes(id: Short): Array[Byte] = { + val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) + encodedBytes + } + + /** + * Function to verify invariants for column family based operations + * such as get, put, remove etc. + * + * @param operationName - name of the store operation + * @param colFamilyName - name of the column family + */ + def verifyColFamilyOperations( + operationName: String, + colFamilyName: String): Unit = { + if (colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME) { + // if the state store instance does not support multiple column families, throw an exception + if (!useColumnFamilies) { + throw StateStoreErrors.unsupportedOperationException(operationName, + multColFamiliesDisabledStr) + } + + // if the column family name is empty or contains leading/trailing whitespaces, throw an + // exception + if (colFamilyName.isEmpty || colFamilyName.trim != colFamilyName) { + throw StateStoreErrors.cannotUseColumnFamilyWithInvalidName(operationName, colFamilyName) + } + + // if the column family does not exist, throw an exception + if (!checkColFamilyExists(colFamilyName)) { + throw StateStoreErrors.unsupportedOperationOnMissingColumnFamily(operationName, + colFamilyName) + } + } + } + + /** + * Function to verify invariants for column family creation or deletion operations. + * + * @param operationName - name of the store operation + * @param colFamilyName - name of the column family + */ + private def verifyColFamilyCreationOrDeletion( + operationName: String, + colFamilyName: String, + isInternal: Boolean = false): Unit = { + // if the state store instance does not support multiple column families, throw an exception + if (!useColumnFamilies) { + throw StateStoreErrors.unsupportedOperationException(operationName, + multColFamiliesDisabledStr) + } + + // if the column family name is empty or contains leading/trailing whitespaces + // or using the reserved "default" column family, throw an exception + if (colFamilyName.isEmpty + || colFamilyName.trim != colFamilyName + || colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) { + throw StateStoreErrors.cannotUseColumnFamilyWithInvalidName(operationName, colFamilyName) + } + + // if the column family is not internal and uses reserved characters, throw an exception + if (!isInternal && colFamilyName.charAt(0) == '_') { + throw StateStoreErrors.cannotCreateColumnFamilyWithReservedChars(colFamilyName) + } + } + + /** + * 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 + */ + def checkInternalColumnFamilies(cfName: String): Boolean = cfName.charAt(0) == '_' + + /** + * Create RocksDB column family, if not created already + */ + def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean = false): Unit = { + verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) + if (!checkColFamilyExists(colFamilyName)) { + colFamilyToIdMap.putIfAbsent(colFamilyName, colFamilyId.incrementAndGet().toShort) + } + } + + /** + * Remove RocksDB column family, if exists + */ + def removeColFamilyIfExists(colFamilyName: String): Boolean = { + verifyColFamilyCreationOrDeletion("remove_col_family", colFamilyName) + if (checkColFamilyExists(colFamilyName)) { + colFamilyToIdMap.remove(colFamilyName) + true + } else { + false + } + } + + /** + * Function to check if the column family exists in the state store instance. + * + * @param colFamilyName - name of the column family + * @return - true if the column family exists, false otherwise + */ + def checkColFamilyExists(colFamilyName: String): Boolean = { + colFamilyToIdMap.keys.asScala.toSeq.contains(colFamilyName) + } + } } object RocksDBStateStoreProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala index 23f867d3e6c07..a35350c505c7e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreChangelog.scala @@ -90,13 +90,9 @@ abstract class StateStoreChangelogWriter( def put(key: Array[Byte], value: Array[Byte]): Unit - def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit - def delete(key: Array[Byte]): Unit - def delete(key: Array[Byte], colFamilyName: String): Unit - - def merge(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit + def merge(key: Array[Byte], value: Array[Byte]): Unit def abort(): Unit = { try { @@ -144,11 +140,6 @@ class StateStoreChangelogWriterV1( size += 1 } - override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = { - throw StateStoreErrors.unsupportedOperationException( - operationName = "Put", entity = "changelog writer v1") - } - override def delete(key: Array[Byte]): Unit = { assert(compressedStream != null) compressedStream.writeInt(key.size) @@ -158,12 +149,7 @@ class StateStoreChangelogWriterV1( size += 1 } - override def delete(key: Array[Byte], colFamilyName: String): Unit = { - throw StateStoreErrors.unsupportedOperationException( - operationName = "Delete", entity = "changelog writer v1") - } - - override def merge(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = { + override def merge(key: Array[Byte], value: Array[Byte]): Unit = { throw new UnsupportedOperationException("Operation not supported with state " + "changelog writer v1") } @@ -202,38 +188,25 @@ class StateStoreChangelogWriterV2( extends StateStoreChangelogWriter(fm, file, compressionCodec) { override def put(key: Array[Byte], value: Array[Byte]): Unit = { - throw StateStoreErrors.unsupportedOperationException( - operationName = "Put", entity = "changelog writer v2") - } - - override def put(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = { - writePutOrMergeRecord(key, value, colFamilyName, RecordType.PUT_RECORD) + writePutOrMergeRecord(key, value, RecordType.PUT_RECORD) } override def delete(key: Array[Byte]): Unit = { - throw StateStoreErrors.unsupportedOperationException( - operationName = "Delete", entity = "changelog writer v2") - } - - override def delete(key: Array[Byte], colFamilyName: String): Unit = { assert(compressedStream != null) compressedStream.write(RecordType.getRecordTypeAsByte(RecordType.DELETE_RECORD)) compressedStream.writeInt(key.size) compressedStream.write(key) // -1 in the value field means record deletion. compressedStream.writeInt(-1) - compressedStream.writeInt(colFamilyName.getBytes.size) - compressedStream.write(colFamilyName.getBytes) size += 1 } - override def merge(key: Array[Byte], value: Array[Byte], colFamilyName: String): Unit = { - writePutOrMergeRecord(key, value, colFamilyName, RecordType.MERGE_RECORD) + override def merge(key: Array[Byte], value: Array[Byte]): Unit = { + writePutOrMergeRecord(key, value, RecordType.MERGE_RECORD) } private def writePutOrMergeRecord(key: Array[Byte], value: Array[Byte], - colFamilyName: String, recordType: RecordType): Unit = { assert(recordType == RecordType.PUT_RECORD || recordType == RecordType.MERGE_RECORD) assert(compressedStream != null) @@ -242,8 +215,6 @@ class StateStoreChangelogWriterV2( compressedStream.write(key) compressedStream.writeInt(value.size) compressedStream.write(value) - compressedStream.writeInt(colFamilyName.getBytes.size) - compressedStream.write(colFamilyName.getBytes) size += 1 } @@ -274,7 +245,7 @@ abstract class StateStoreChangelogReader( fm: CheckpointFileManager, fileToRead: Path, compressionCodec: CompressionCodec) - extends NextIterator[(RecordType.Value, Array[Byte], Array[Byte], String)] with Logging { + extends NextIterator[(RecordType.Value, Array[Byte], Array[Byte])] with Logging { private def decompressStream(inputStream: DataInputStream): DataInputStream = { val compressed = compressionCodec.compressedInputStream(inputStream) @@ -291,7 +262,7 @@ abstract class StateStoreChangelogReader( def close(): Unit = { if (input != null) input.close() } - override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) + override def getNext(): (RecordType.Value, Array[Byte], Array[Byte]) } /** @@ -307,7 +278,7 @@ class StateStoreChangelogReaderV1( compressionCodec: CompressionCodec) extends StateStoreChangelogReader(fm, fileToRead, compressionCodec) { - override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = { + override def getNext(): (RecordType.Value, Array[Byte], Array[Byte]) = { val keySize = input.readInt() // A -1 key size mean end of file. if (keySize == -1) { @@ -323,12 +294,12 @@ class StateStoreChangelogReaderV1( val valueSize = input.readInt() if (valueSize < 0) { // A deletion record - (RecordType.DELETE_RECORD, keyBuffer, null, StateStore.DEFAULT_COL_FAMILY_NAME) + (RecordType.DELETE_RECORD, keyBuffer, null) } else { val valueBuffer = new Array[Byte](valueSize) ByteStreams.readFully(input, valueBuffer, 0, valueSize) // A put record. - (RecordType.PUT_RECORD, keyBuffer, valueBuffer, StateStore.DEFAULT_COL_FAMILY_NAME) + (RecordType.PUT_RECORD, keyBuffer, valueBuffer) } } } @@ -354,7 +325,7 @@ class StateStoreChangelogReaderV2( blockBuffer } - override def getNext(): (RecordType.Value, Array[Byte], Array[Byte], String) = { + override def getNext(): (RecordType.Value, Array[Byte], Array[Byte]) = { val recordType = RecordType.getRecordTypeFromByte(input.readByte()) // A EOF_RECORD means end of file. if (recordType == RecordType.EOF_RECORD) { @@ -365,24 +336,18 @@ class StateStoreChangelogReaderV2( case RecordType.PUT_RECORD => val keyBuffer = parseBuffer(input) val valueBuffer = parseBuffer(input) - val colFamilyNameBuffer = parseBuffer(input) - (RecordType.PUT_RECORD, keyBuffer, valueBuffer, - colFamilyNameBuffer.map(_.toChar).mkString) + (RecordType.PUT_RECORD, keyBuffer, valueBuffer) case RecordType.DELETE_RECORD => val keyBuffer = parseBuffer(input) val valueSize = input.readInt() assert(valueSize == -1) - val colFamilyNameBuffer = parseBuffer(input) - (RecordType.DELETE_RECORD, keyBuffer, null, - colFamilyNameBuffer.map(_.toChar).mkString) + (RecordType.DELETE_RECORD, keyBuffer, null) case RecordType.MERGE_RECORD => val keyBuffer = parseBuffer(input) val valueBuffer = parseBuffer(input) - val colFamilyNameBuffer = parseBuffer(input) - (RecordType.MERGE_RECORD, keyBuffer, valueBuffer, - colFamilyNameBuffer.map(_.toChar).mkString) + (RecordType.MERGE_RECORD, keyBuffer, valueBuffer) case _ => throw new IOException("Failed to process unknown record type") 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 1c7343badcaeb..450e622ede263 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 @@ -977,6 +977,9 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, keyEncoder) + // remove non-exist col family will return false + assert(!store.removeColFamilyIfExists("non-existence")) + // put some test data into state store val timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, -230L, -14569L, -92L, -7434253L, 35L, 6L, 9L, -323L, 5L) @@ -998,8 +1001,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid exception = e.asInstanceOf[StateStoreUnsupportedOperationOnMissingColumnFamily], errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", sqlState = Some("42802"), - // TODO how to throw error with iterator? - parameters = Map("operationType" -> "prefixScan", "colFamilyName" -> cfName) + parameters = Map("operationType" -> "iterator", "colFamilyName" -> cfName) ) } } 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 6086fd43846f6..0bd9df9a909f9 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 @@ -14,31 +14,34 @@ * See the License for the specific language governing permissions and * limitations under the License. */ - +// scalastyle:off package org.apache.spark.sql.execution.streaming.state import java.io._ import java.nio.charset.Charset -import scala.collection.mutable +// import scala.collection.mutable import scala.language.implicitConversions import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.rocksdb.CompressionType +// import org.rocksdb.CompressionType import org.scalactic.source.Position import org.scalatest.Tag -import org.apache.spark.{SparkException, SparkUnsupportedOperationException} -import org.apache.spark.sql.catalyst.util.quietly -import org.apache.spark.sql.execution.streaming.{CreateAtomicTestManager, FileSystemBasedCheckpointFileManager} +// import org.apache.spark.{SparkException, SparkUnsupportedOperationException} +// import org.apache.spark.sql.catalyst.util.quietly +// import org.apache.spark.sql.execution.streaming. +// {CreateAtomicTestManager, FileSystemBasedCheckpointFileManager} +import org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager import org.apache.spark.sql.execution.streaming.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS +// import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.tags.SlowSQLTest -import org.apache.spark.util.{ThreadUtils, Utils} +// import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.Utils import org.apache.spark.util.ArrayImplicits._ class NoOverwriteFileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration) @@ -164,7 +167,7 @@ trait AlsoTestWithChangelogCheckpointingEnabled class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with SharedSparkSession { sqlConf.setConf(SQLConf.STATE_STORE_PROVIDER_CLASS, classOf[RocksDBStateStoreProvider].getName) - +/* testWithColumnFamilies( "RocksDB: check changelog and snapshot version", TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => @@ -2221,7 +2224,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } } - } + } */ private def sqlConf = SQLConf.get.clone() @@ -2242,8 +2245,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared conf = conf, localRootDir = localDir, hadoopConf = hadoopConf, - loggingId = s"[Thread-${Thread.currentThread.getId}]", - useColumnFamilies = useColumnFamilies + loggingId = s"[Thread-${Thread.currentThread.getId}]" ) db.load(version) func(db) @@ -2293,8 +2295,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared def toStr(kv: ByteArrayPair): (String, String) = (toStr(kv.key), toStr(kv.value)) - def iterator(db: RocksDB, colFamilyName: String = "default"): - Iterator[(String, String)] = db.iterator(colFamilyName).map(toStr) + def iterator(db: RocksDB): + Iterator[(String, String)] = db.iterator().map(toStr) def listFiles(file: File): Seq[File] = { if (!file.exists()) return Seq.empty @@ -2325,3 +2327,5 @@ object RocksDBSuite { } } } + +// scalastyle:on diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 0057af44d3e37..79214dc78cbd3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -573,16 +573,18 @@ class TransformWithStateSuite extends StateStoreMetricsTest TimeMode.None(), OutputMode.Update()) + /* testStream(stream1, OutputMode.Update())( StartStream(checkpointLocation = dirPath), AddData(inputData, ("a", "str1")), CheckNewAnswer(("a", "1", "")), StopStream - ) + ) */ testStream(stream2, OutputMode.Update())( StartStream(checkpointLocation = dirPath), + AddData(inputData, ("a", "str1")), AddData(inputData, ("a", "str2"), ("b", "str3")), - CheckNewAnswer(("a", "str1"), + CheckNewAnswer(("a", ""), ("a", "str1"), ("b", "")), // should not factor in previous count state Execute { q => assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0) From 053b6963224fcf674e170de9a2eda61ba3a66b2a Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 14:11:50 -0700 Subject: [PATCH 13/23] fix all suites --- .../StatefulProcessorHandleImpl.scala | 1 - .../state/RocksDBStateStoreProvider.scala | 39 +- .../state/RocksDBStateStoreSuite.scala | 212 ++++++++- .../streaming/state/RocksDBSuite.scala | 439 ++---------------- .../streaming/TransformWithStateSuite.scala | 9 +- 5 files changed, 262 insertions(+), 438 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala index 90d823d17dd7b..dcc77e94de280 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala @@ -130,7 +130,6 @@ class StatefulProcessorHandleImpl( valEncoder: Encoder[T]): ValueState[T] = { verifyStateVarOperations("get_value_state") incrementMetric("numValueStateVars") - println("I am inside getValueState, after inc metric: " + metrics.get("numValueStateVars")) val resultState = new ValueStateImpl[T](store, stateName, keyEncoder, valEncoder) resultState } 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 43a4ef107cb2f..febbb0900c88c 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 @@ -59,9 +59,6 @@ private[sql] class RocksDBStateStoreProvider keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, isInternal: Boolean = false): Unit = { - verify(colFamilyName != StateStore.DEFAULT_COL_FAMILY_NAME, - s"Failed to create column family with reserved_name=$colFamilyName") - verify(useColumnFamilies, "Column families are not supported in this store") ColumnFamilyUtils.createColFamilyIfAbsent(colFamilyName, isInternal) keyValueEncoderMap.putIfAbsent(colFamilyName, @@ -71,10 +68,10 @@ private[sql] class RocksDBStateStoreProvider override def get(key: UnsafeRow, colFamilyName: String): UnsafeRow = { verify(key != null, "Key cannot be null") - val kvEncoder = keyValueEncoderMap.get(colFamilyName) + ColumnFamilyUtils.verifyColFamilyOperations("get", colFamilyName) + val kvEncoder = keyValueEncoderMap.get(colFamilyName) val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) - ColumnFamilyUtils.verifyColFamilyOperations("get", colFamilyName) val value = kvEncoder._2.decodeValue(rocksDB.get(encodedKey)) @@ -98,6 +95,7 @@ private[sql] class RocksDBStateStoreProvider */ override def valuesIterator(key: UnsafeRow, colFamilyName: String): Iterator[UnsafeRow] = { verify(key != null, "Key cannot be null") + ColumnFamilyUtils.verifyColFamilyOperations("valuesIterator", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) val valueEncoder = kvEncoder._2 @@ -107,7 +105,6 @@ private[sql] class RocksDBStateStoreProvider "that supports multiple values for a single key.") val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) - ColumnFamilyUtils.verifyColFamilyOperations("get", colFamilyName) val encodedValues = rocksDB.get(encodedKey) valueEncoder.decodeValues(encodedValues) } @@ -115,6 +112,7 @@ private[sql] class RocksDBStateStoreProvider override def merge(key: UnsafeRow, value: UnsafeRow, colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { verify(state == UPDATING, "Cannot merge after already committed or aborted") + ColumnFamilyUtils.verifyColFamilyOperations("merge", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) val keyEncoder = kvEncoder._1 val valueEncoder = kvEncoder._2 @@ -122,9 +120,8 @@ private[sql] class RocksDBStateStoreProvider " which supports multiple values for a single key") verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) - ColumnFamilyUtils.verifyColFamilyOperations("merge", colFamilyName) + val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) rocksDB.merge(encodedKey, valueEncoder.encodeValue(value)) } @@ -132,32 +129,33 @@ private[sql] class RocksDBStateStoreProvider verify(state == UPDATING, "Cannot put after already committed or aborted") verify(key != null, "Key cannot be null") require(value != null, "Cannot put a null value") - val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) ColumnFamilyUtils.verifyColFamilyOperations("put", colFamilyName) + val kvEncoder = keyValueEncoderMap.get(colFamilyName) + val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) rocksDB.put(encodedKey, kvEncoder._2.encodeValue(value)) } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { verify(state == UPDATING, "Cannot remove after already committed or aborted") verify(key != null, "Key cannot be null") + ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) + val kvEncoder = keyValueEncoderMap.get(colFamilyName) val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) - - ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) rocksDB.remove(encodedKey) } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { + // Note this verify function only verify on the colFamilyName being valid, + // we are actually doing prefix when useColumnFamilies, + // but pass "iterator" to throw correct error message + ColumnFamilyUtils.verifyColFamilyOperations("iterator", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() if (useColumnFamilies) { val cfId: Short = colFamilyToIdMap.get(colFamilyName) - // Note this verify function only verify on the colFamilyName being valid, - // we are actually doing prefix, but pass "iterator" to throw correct error message - ColumnFamilyUtils.verifyColFamilyOperations("iterator", colFamilyName) rocksDB.prefixScan(ColumnFamilyUtils.getVcfIdBytes(cfId)).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), kvEncoder._2.decodeValue(kv.value)) @@ -169,7 +167,6 @@ private[sql] class RocksDBStateStoreProvider rowPair } } else { - ColumnFamilyUtils.verifyColFamilyOperations("iterator", colFamilyName) rocksDB.iterator().map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) @@ -185,15 +182,15 @@ private[sql] class RocksDBStateStoreProvider override def prefixScan(prefixKey: UnsafeRow, colFamilyName: String): Iterator[UnsafeRowPair] = { + ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) require(kvEncoder._1.supportPrefixKeyScan, "Prefix scan requires setting prefix key!") val rowPair = new UnsafeRowPair() + val prefix = kvEncoder._1.encodePrefixKey(prefixKey, Option(colFamilyToIdMap.get(colFamilyName))) - - ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) rocksDB.prefixScan(prefix).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, useColumnFamilies), kvEncoder._2.decodeValue(kv.value)) @@ -313,12 +310,12 @@ private[sql] class RocksDBStateStoreProvider override def removeColFamilyIfExists(colFamilyName: String): Boolean = { verify(useColumnFamilies, "Column families are not supported in this store") val result = { - val idPrefix = ColumnFamilyUtils.getVcfIdBytes( - colFamilyToIdMap.get(colFamilyName) - ) val colFamilyExists = ColumnFamilyUtils.checkColFamilyExists(colFamilyName) if (colFamilyExists) { ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) + val idPrefix = ColumnFamilyUtils.getVcfIdBytes( + colFamilyToIdMap.get(colFamilyName) + ) rocksDB.prefixScan(idPrefix).foreach { kv => ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) rocksDB.remove(kv.key) 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 450e622ede263..01865ec8ac488 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 @@ -29,7 +29,7 @@ import org.apache.spark.{SparkConf, SparkUnsupportedOperationException} import org.apache.spark.io.CompressionCodec import org.apache.spark.sql.LocalSparkSession.withSparkSession import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.execution.streaming.StatefulOperatorStateInfo import org.apache.spark.sql.internal.SQLConf @@ -918,6 +918,178 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } } + /* Column family related tests */ + testWithColumnFamilies("column family creation with invalid names", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + tryWithProviderResource( + newStoreProvider(useColumnFamilies = colFamiliesEnabled)) { provider => + val store = provider.getStore(0) + + Seq("default", "", " ", " ", " default", " default ").foreach { colFamilyName => + val ex = intercept[SparkUnsupportedOperationException] { + store.createColFamilyIfAbsent(colFamilyName, + keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) + } + + if (!colFamiliesEnabled) { + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + parameters = Map( + "operationType" -> "create_col_family", + "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + ), + matchPVals = true + ) + } else { + checkError( + ex, + errorClass = "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME", + parameters = Map( + "operationName" -> "create_col_family", + "colFamilyName" -> colFamilyName + ), + matchPVals = true + ) + } + } + } + } + + testWithColumnFamilies(s"column family creation with reserved chars", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + tryWithProviderResource( + newStoreProvider(useColumnFamilies = colFamiliesEnabled)) { provider => + val store = provider.getStore(0) + + Seq("_internal", "_test", "_test123", "__12345").foreach { colFamilyName => + val ex = intercept[SparkUnsupportedOperationException] { + store.createColFamilyIfAbsent(colFamilyName, + keySchema, valueSchema, NoPrefixKeyStateEncoderSpec(keySchema)) + } + + if (!colFamiliesEnabled) { + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + parameters = Map( + "operationType" -> "create_col_family", + "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + ), + matchPVals = true + ) + } else { + checkError( + ex, + errorClass = "STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS", + parameters = Map( + "colFamilyName" -> colFamilyName + ), + matchPVals = true + ) + } + } + } + } + + testWithColumnFamilies(s"operations on absent column family", + TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => + tryWithProviderResource( + newStoreProvider(useColumnFamilies = colFamiliesEnabled)) { provider => + val store = provider.getStore(0) + + val colFamilyName = "test" + + verifyStoreOperationUnsupported("put", colFamiliesEnabled, colFamilyName) { + store.put(dataToKeyRow("a", 1), dataToValueRow(1), colFamilyName) + } + + verifyStoreOperationUnsupported("remove", colFamiliesEnabled, colFamilyName) { + store.remove(dataToKeyRow("a", 1), colFamilyName) + } + + verifyStoreOperationUnsupported("get", colFamiliesEnabled, colFamilyName) { + store.get(dataToKeyRow("a", 1), colFamilyName) + } + + verifyStoreOperationUnsupported("iterator", colFamiliesEnabled, colFamilyName) { + store.iterator(colFamilyName) + } + + verifyStoreOperationUnsupported("merge", colFamiliesEnabled, colFamilyName) { + store.merge(dataToKeyRow("a", 1), dataToValueRow(1), colFamilyName) + } + + verifyStoreOperationUnsupported("prefixScan", colFamiliesEnabled, colFamilyName) { + store.prefixScan(dataToKeyRow("a", 1), colFamilyName) + } + } + } + + // TODO SPARK-48796 after restart state id will not be the same + ignore(s"get, put, iterator, commit, load with multiple column families") { + tryWithProviderResource(newStoreProvider(useColumnFamilies = true)) { provider => + def get(store: StateStore, col1: String, col2: Int, colFamilyName: String): UnsafeRow = { + store.get(dataToKeyRow(col1, col2), colFamilyName) + } + + def iterator(store: StateStore, colFamilyName: String): Seq[((String, Int), Int)] = { + store.iterator(colFamilyName).toSeq.map { + case unsafePair => + (keyRowToData(unsafePair.key), valueRowToData(unsafePair.value)) + } + } + + def put(store: StateStore, key: (String, Int), value: Int, colFamilyName: String): Unit = { + store.put(dataToKeyRow(key._1, key._2), dataToValueRow(value), colFamilyName) + } + + var store = provider.getStore(0) + + val colFamily1: String = "abc" + val colFamily2: String = "xyz" + store.createColFamilyIfAbsent(colFamily1, keySchema, valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema)) + store.createColFamilyIfAbsent(colFamily2, keySchema, valueSchema, + NoPrefixKeyStateEncoderSpec(keySchema)) + + assert(get(store, "a", 1, colFamily1) === null) + assert(iterator(store, colFamily1).isEmpty) + put(store, ("a", 1), 1, colFamily1) + assert(valueRowToData(get(store, "a", 1, colFamily1)) === 1) + + assert(get(store, "a", 1, colFamily2) === null) + assert(iterator(store, colFamily2).isEmpty) + put(store, ("a", 1), 1, colFamily2) + assert(valueRowToData(get(store, "a", 1, colFamily2)) === 1) + + store.commit() + + // reload version 0 + store = provider.getStore(0) + assert(get(store, "a", 1, colFamily1) === null) + assert(iterator(store, colFamily1).isEmpty) + + store = provider.getStore(1) + // version 1 data recovered correctly + assert(valueRowToData(get(store, "a", 1, colFamily1)) == 1) + assert(iterator(store, colFamily1).toSet === Set((("a", 1), 1))) + // make changes but do not commit version 2 + put(store, ("b", 1), 2, colFamily1) + assert(valueRowToData(get(store, "b", 1, colFamily1)) === 2) + assert(iterator(store, colFamily1).toSet === Set((("a", 1), 1), (("b", 1), 2))) + // version 1 data recovered correctly + assert(valueRowToData(get(store, "a", 1, colFamily2))== 1) + assert(iterator(store, colFamily2).toSet === Set((("a", 1), 1))) + // make changes but do not commit version 2 + put(store, ("b", 1), 2, colFamily2) + assert(valueRowToData(get(store, "b", 1, colFamily2))=== 2) + assert(iterator(store, colFamily2).toSet === Set((("a", 1), 1), (("b", 1), 2))) + + store.commit() + } + } + Seq( NoPrefixKeyStateEncoderSpec(keySchema), PrefixKeyScanStateEncoderSpec(keySchema, 1) ).foreach { keyEncoder => @@ -930,8 +1102,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid val cfName = if (colFamiliesEnabled) "testColFamily" else "default" if (colFamiliesEnabled) { - store.createColFamilyIfAbsent(cfName, - keySchema, valueSchema, keyEncoder) + store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, keyEncoder) } var timerTimestamps = Seq(931L, 8000L, 452300L, 4200L, -1L, 90L, 1L, 2L, 8L, @@ -974,8 +1145,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid val store = provider.getStore(0) val cfName = "testColFamily" - store.createColFamilyIfAbsent(cfName, - keySchema, valueSchema, keyEncoder) + store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, keyEncoder) // remove non-exist col family will return false assert(!store.removeColFamilyIfExists("non-existence")) @@ -1007,6 +1177,38 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid } } + private def verifyStoreOperationUnsupported( + operationName: String, + colFamiliesEnabled: Boolean, + colFamilyName: String) + (testFn: => Unit): Unit = { + val ex = intercept[SparkUnsupportedOperationException] { + testFn + } + + if (!colFamiliesEnabled) { + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", + parameters = Map( + "operationType" -> operationName, + "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + ), + matchPVals = true + ) + } else { + checkError( + ex, + errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", + parameters = Map( + "operationType" -> operationName, + "colFamilyName" -> colFamilyName + ), + matchPVals = true + ) + } + } + override def newStoreProvider(): RocksDBStateStoreProvider = { newStoreProvider(StateStoreId(newDir(), Random.nextInt(), 0)) } 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 0bd9df9a909f9..b47a936d516c4 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 @@ -14,34 +14,31 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -// scalastyle:off + package org.apache.spark.sql.execution.streaming.state import java.io._ import java.nio.charset.Charset -// import scala.collection.mutable +import scala.collection.mutable import scala.language.implicitConversions import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -// import org.rocksdb.CompressionType +import org.rocksdb.CompressionType import org.scalactic.source.Position import org.scalatest.Tag -// import org.apache.spark.{SparkException, SparkUnsupportedOperationException} -// import org.apache.spark.sql.catalyst.util.quietly -// import org.apache.spark.sql.execution.streaming. -// {CreateAtomicTestManager, FileSystemBasedCheckpointFileManager} -import org.apache.spark.sql.execution.streaming.FileSystemBasedCheckpointFileManager +import org.apache.spark.SparkException +import org.apache.spark.sql.catalyst.util.quietly +import org.apache.spark.sql.execution.streaming.{CreateAtomicTestManager, FileSystemBasedCheckpointFileManager} import org.apache.spark.sql.execution.streaming.CheckpointFileManager.{CancellableFSDataOutputStream, RenameBasedFSDataOutputStream} import org.apache.spark.sql.internal.SQLConf -// import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS +import org.apache.spark.sql.internal.SQLConf.STREAMING_CHECKPOINT_FILE_MANAGER_CLASS import org.apache.spark.sql.test.{SharedSparkSession, SQLTestUtils} import org.apache.spark.tags.SlowSQLTest -// import org.apache.spark.util.{ThreadUtils, Utils} -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.util.ArrayImplicits._ class NoOverwriteFileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration) @@ -167,7 +164,7 @@ trait AlsoTestWithChangelogCheckpointingEnabled class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with SharedSparkSession { sqlConf.setConf(SQLConf.STATE_STORE_PROVIDER_CLASS, classOf[RocksDBStateStoreProvider].getName) -/* + testWithColumnFamilies( "RocksDB: check changelog and snapshot version", TestWithChangelogCheckpointingEnabled) { colFamiliesEnabled => @@ -547,264 +544,6 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - testWithColumnFamilies(s"RocksDB: column family creation with invalid names", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - - val conf = RocksDBConf().copy() - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - Seq("default", "", " ", " ", " default", " default ").foreach { colFamilyName => - val ex = intercept[SparkUnsupportedOperationException] { - db.createColFamilyIfAbsent(colFamilyName) - } - - if (!colFamiliesEnabled) { - checkError( - ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", - parameters = Map( - "operationType" -> "create_col_family", - "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" - ), - matchPVals = true - ) - } else { - checkError( - ex, - errorClass = "STATE_STORE_CANNOT_USE_COLUMN_FAMILY_WITH_INVALID_NAME", - parameters = Map( - "operationName" -> "create_col_family", - "colFamilyName" -> colFamilyName - ), - matchPVals = true - ) - } - } - } - } - - testWithColumnFamilies(s"RocksDB: column family creation with reserved chars", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - - val conf = RocksDBConf().copy() - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - Seq("_internal", "_test", "_test123", "__12345").foreach { colFamilyName => - val ex = intercept[SparkUnsupportedOperationException] { - db.createColFamilyIfAbsent(colFamilyName) - } - - if (!colFamiliesEnabled) { - checkError( - ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", - parameters = Map( - "operationType" -> "create_col_family", - "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" - ), - matchPVals = true - ) - } else { - checkError( - ex, - errorClass = "STATE_STORE_CANNOT_CREATE_COLUMN_FAMILY_WITH_RESERVED_CHARS", - parameters = Map( - "colFamilyName" -> colFamilyName - ), - matchPVals = true - ) - } - } - } - } - - - private def verifyStoreOperationUnsupported( - operationName: String, - colFamiliesEnabled: Boolean, - colFamilyName: String) - (testFn: => Unit): Unit = { - val ex = intercept[SparkUnsupportedOperationException] { - testFn - } - - if (!colFamiliesEnabled) { - checkError( - ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", - parameters = Map( - "operationType" -> operationName, - "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" - ), - matchPVals = true - ) - } else { - checkError( - ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", - parameters = Map( - "operationType" -> operationName, - "colFamilyName" -> colFamilyName - ), - matchPVals = true - ) - } - } - - testWithColumnFamilies(s"RocksDB: operations on absent column family", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - - val conf = RocksDBConf().copy() - withDB(remoteDir, conf = conf, useColumnFamilies = colFamiliesEnabled) { db => - db.load(0) - val colFamilyName = "test" - verifyStoreOperationUnsupported("put", colFamiliesEnabled, colFamilyName) { - db.put("a", "1", colFamilyName) - } - - verifyStoreOperationUnsupported("remove", colFamiliesEnabled, colFamilyName) { - db.remove("a", colFamilyName) - } - - verifyStoreOperationUnsupported("get", colFamiliesEnabled, colFamilyName) { - db.get("a", colFamilyName) - } - - verifyStoreOperationUnsupported("iterator", colFamiliesEnabled, colFamilyName) { - db.iterator(colFamilyName) - } - - verifyStoreOperationUnsupported("merge", colFamiliesEnabled, colFamilyName) { - db.merge("a", "1", colFamilyName) - } - - verifyStoreOperationUnsupported("prefixScan", colFamiliesEnabled, colFamilyName) { - db.prefixScan("a", colFamilyName) - } - } - } - - testWithColumnFamilies(s"RocksDB: get, put, iterator, commit, load " + - s"with multiple column families", - TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => - val remoteDir = Utils.createTempDir().toString - new File(remoteDir).delete() // to make sure that the directory gets created - val colFamily1: String = "abc" - val colFamily2: String = "xyz" - - val conf = RocksDBConf().copy() - withDB(remoteDir, conf = conf, useColumnFamilies = true) { db => - db.createColFamilyIfAbsent(colFamily1) - db.createColFamilyIfAbsent(colFamily2) - - assert(db.get("a", colFamily1) === null) - assert(iterator(db, colFamily1).isEmpty) - - db.put("a", "1", colFamily1) - assert(toStr(db.get("a", colFamily1)) === "1") - - assert(db.get("a", colFamily2) === null) - assert(iterator(db, colFamily2).isEmpty) - - db.put("a", "1", colFamily2) - assert(toStr(db.get("a", colFamily2)) === "1") - - db.commit() - } - - withDB(remoteDir, conf = conf, version = 0, useColumnFamilies = true) { db => - val ex = intercept[SparkUnsupportedOperationException] { - // version 0 can be loaded again - assert(toStr(db.get("a", colFamily1)) === null) - assert(iterator(db, colFamily1).isEmpty) - - // version 0 can be loaded again - assert(toStr(db.get("a", colFamily2)) === null) - assert(iterator(db, colFamily2).isEmpty) - } - checkError( - ex, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION_ON_MISSING_COLUMN_FAMILY", - parameters = Map( - "operationType" -> "get", - "colFamilyName" -> colFamily1 - ), - matchPVals = true - ) - } - - withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = true) { db => - // version 1 data recovered correctly - assert(toStr(db.get("a", colFamily1)) === "1") - assert(db.iterator(colFamily1).map(toStr).toSet === Set(("a", "1"))) - - // make changes but do not commit version 2 - db.put("b", "2", colFamily1) - assert(toStr(db.get("b", colFamily1)) === "2") - assert(db.iterator(colFamily1).map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - - // version 1 data recovered correctly - assert(toStr(db.get("a", colFamily2)) === "1") - assert(db.iterator(colFamily2).map(toStr).toSet === Set(("a", "1"))) - - // make changes but do not commit version 2 - db.put("b", "2", colFamily2) - assert(toStr(db.get("b", colFamily2)) === "2") - assert(db.iterator(colFamily2).map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - } - - withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = true) { db => - // version 1 data not changed - assert(toStr(db.get("a", colFamily1)) === "1") - assert(db.get("b", colFamily1) === null) - assert(db.iterator(colFamily1).map(toStr).toSet === Set(("a", "1"))) - - assert(toStr(db.get("a", colFamily2)) === "1") - assert(db.get("b", colFamily2) === null) - assert(db.iterator(colFamily2).map(toStr).toSet === Set(("a", "1"))) - - // commit version 2 - db.put("b", "2", colFamily1) - assert(toStr(db.get("b", colFamily1)) === "2") - assert(db.iterator(colFamily1).map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - - db.put("b", "2", colFamily2) - assert(toStr(db.get("b", colFamily2)) === "2") - db.commit() - assert(db.iterator(colFamily2).map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - } - - withDB(remoteDir, conf = conf, version = 1, useColumnFamilies = true) { db => - // version 1 data not changed - assert(toStr(db.get("a", colFamily1)) === "1") - assert(db.get("b", colFamily1) === null) - - assert(toStr(db.get("a", colFamily2)) === "1") - assert(db.get("b", colFamily2) === null) - } - - withDB(remoteDir, conf = conf, version = 2, useColumnFamilies = true) { db => - // version 2 can be loaded again - assert(toStr(db.get("b", colFamily1)) === "2") - assert(db.iterator(colFamily1).map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - - assert(toStr(db.get("b", colFamily2)) === "2") - assert(db.iterator(colFamily2).map(toStr).toSet === Set(("a", "1"), ("b", "2"))) - - db.load(1) - assert(toStr(db.get("b", colFamily1)) === null) - assert(db.iterator(colFamily1).map(toStr).toSet === Set(("a", "1"))) - - db.load(1) - assert(toStr(db.get("b", colFamily2)) === null) - assert(db.iterator(colFamily2).map(toStr).toSet === Set(("a", "1"))) - } - } - testWithColumnFamilies(s"RocksDB: handle commit failures and aborts", TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => val hadoopConf = new Configuration() @@ -849,32 +588,11 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration) val changelogWriter = fileManager.getChangeLogWriter(1) - val ex1 = intercept[SparkUnsupportedOperationException] { - changelogWriter.put("a", "1", "testColFamily") + val ex = intercept[UnsupportedOperationException] { + changelogWriter.merge("a", "1") } - checkError( - ex1, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", - parameters = Map( - "operationType" -> "Put", - "entity" -> "changelog writer v1" - ), - matchPVals = true - ) - val ex2 = intercept[SparkUnsupportedOperationException] { - changelogWriter.delete("a", "testColFamily") - } - - checkError( - ex2, - errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", - parameters = Map( - "operationType" -> "Delete", - "entity" -> "changelog writer v1" - ), - matchPVals = true - ) + assert(ex.getMessage.contains("state changelog writer v1")) } testWithChangelogCheckpointingEnabled("RocksDBFileManager: read and write changelog") { @@ -899,8 +617,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ - case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && - e1._3 === e2._3 && e1._4 === e2._4) + case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) } } @@ -911,83 +628,30 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration) val changelogWriter = fileManager.getChangeLogWriter(1, true) (1 to 5).foreach { i => - changelogWriter.put(i.toString, i.toString, StateStore.DEFAULT_COL_FAMILY_NAME) + changelogWriter.put(i.toString, i.toString) } (1 to 5).foreach { i => - changelogWriter.merge(i.toString, i.toString, StateStore.DEFAULT_COL_FAMILY_NAME) + changelogWriter.merge(i.toString, i.toString) } (2 to 4).foreach { j => - changelogWriter.delete(j.toString, StateStore.DEFAULT_COL_FAMILY_NAME) + changelogWriter.delete(j.toString) } changelogWriter.commit() val changelogReader = fileManager.getChangelogReader(1, true) val entries = changelogReader.toSeq val expectedEntries = (1 to 5).map { i => - (RecordType.PUT_RECORD, i.toString.getBytes, - i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME) - } ++ (1 to 5).map { i => - (RecordType.MERGE_RECORD, i.toString.getBytes, - i.toString.getBytes, StateStore.DEFAULT_COL_FAMILY_NAME) - } ++ (2 to 4).map { j => - (RecordType.DELETE_RECORD, j.toString.getBytes, - null, StateStore.DEFAULT_COL_FAMILY_NAME) - } - - assert(entries.size == expectedEntries.size) - entries.zip(expectedEntries).map{ - case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && - e1._3 === e2._3 && e1._4 === e2._4) - } - } - - testWithChangelogCheckpointingEnabled( - "RocksDBFileManager: read and write v2 changelog with multiple col families") { - val dfsRootDir = new File(Utils.createTempDir().getAbsolutePath + "/state/1/1") - val testColFamily1: String = "testColFamily1" - val testColFamily2: String = "testColFamily2" - val fileManager = new RocksDBFileManager( - dfsRootDir.getAbsolutePath, Utils.createTempDir(), new Configuration) - val changelogWriter = fileManager.getChangeLogWriter(1, true) - (1 to 5).foreach(i => changelogWriter.put(i.toString, i.toString, testColFamily1)) - (1 to 5).foreach(i => changelogWriter.merge(i.toString, i.toString, testColFamily1)) - (2 to 4).foreach(j => changelogWriter.delete(j.toString, testColFamily1)) - - (1 to 5).foreach(i => changelogWriter.put(i.toString, i.toString, testColFamily2)) - (1 to 5).foreach(i => changelogWriter.merge(i.toString, i.toString, testColFamily2)) - (2 to 4).foreach(j => changelogWriter.delete(j.toString, testColFamily2)) - - changelogWriter.commit() - val changelogReader = fileManager.getChangelogReader(1, true) - val entries = changelogReader.toSeq - val expectedEntriesForColFamily1 = (1 to 5).map { i => - (RecordType.PUT_RECORD, i.toString.getBytes, - i.toString.getBytes, testColFamily1) + (RecordType.PUT_RECORD, i.toString.getBytes, i.toString.getBytes) } ++ (1 to 5).map { i => - (RecordType.MERGE_RECORD, i.toString.getBytes, - i.toString.getBytes, testColFamily1) + (RecordType.MERGE_RECORD, i.toString.getBytes, i.toString.getBytes) } ++ (2 to 4).map { j => - (RecordType.DELETE_RECORD, j.toString.getBytes, - null, testColFamily1) + (RecordType.DELETE_RECORD, j.toString.getBytes, null) } - val expectedEntriesForColFamily2 = (1 to 5).map { i => - (RecordType.PUT_RECORD, i.toString.getBytes, - i.toString.getBytes, testColFamily2) - } ++ (1 to 5).map { i => - (RecordType.MERGE_RECORD, i.toString.getBytes, - i.toString.getBytes, testColFamily2) - } ++ (2 to 4).map { j => - (RecordType.DELETE_RECORD, j.toString.getBytes, - null, testColFamily2) - } - - val expectedEntries = expectedEntriesForColFamily1 ++ expectedEntriesForColFamily2 assert(entries.size == expectedEntries.size) entries.zip(expectedEntries).map{ - case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && - e1._3 === e2._3 && e1._4 === e2._4) + case (e1, e2) => assert(e1._1 === e2._1 && e1._2 === e2._2 && e1._3 === e2._3) } } @@ -1012,21 +676,6 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } - test("ensure merge operation is not supported if column families is not enabled") { - withTempDir { dir => - val remoteDir = Utils.createTempDir().toString - 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 = false) { db => - db.load(0) - db.put("a", "1") - intercept[RuntimeException]( - db.merge("a", "2") - ) - } - } - } - test("RocksDB: ensure merge operation correctness") { withTempDir { dir => val remoteDir = Utils.createTempDir().toString @@ -1036,48 +685,29 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared new File(remoteDir).delete() // to make sure that the directory gets created withDB(remoteDir, conf = conf, useColumnFamilies = true) { db => db.load(0) - db.createColFamilyIfAbsent("cf1") - db.createColFamilyIfAbsent("cf2") - db.put("a", "1", "cf1") - db.merge("a", "2", "cf1") - db.put("a", "3", "cf2") + db.put("a", "1") + db.merge("a", "2") db.commit() db.load(1) - db.put("a", "2") - db.merge("a", "3", "cf1") - db.merge("a", "4", "cf2") + db.merge("a", "3") db.commit() db.load(2) - db.remove("a", "cf1") - db.merge("a", "5") - db.merge("a", "6", "cf2") + db.remove("a") db.commit() db.load(1) - assert(new String(db.get("a", "cf1")) === "1,2") - assert(new String(db.get("a", "cf2")) === "3") - assert(db.get("a") === null) - assert(db.iterator("cf1").map(toStr).toSet === Set(("a", "1,2"))) - assert(db.iterator("cf2").map(toStr).toSet === Set(("a", "3"))) - assert(db.iterator().isEmpty) + assert(new String(db.get("a")) === "1,2") + assert(db.iterator().map(toStr).toSet === Set(("a", "1,2"))) db.load(2) - assert(new String(db.get("a", "cf1")) === "1,2,3") - assert(new String(db.get("a", "cf2")) === "3,4") - assert(new String(db.get("a")) === "2") - assert(db.iterator("cf1").map(toStr).toSet === Set(("a", "1,2,3"))) - assert(db.iterator("cf2").map(toStr).toSet === Set(("a", "3,4"))) - assert(db.iterator().map(toStr).toSet === Set(("a", "2"))) + assert(new String(db.get("a")) === "1,2,3") + assert(db.iterator().map(toStr).toSet === Set(("a", "1,2,3"))) db.load(3) - assert(db.get("a", "cf1") === null) - assert(new String(db.get("a", "cf2")) === "3,4,6") - assert(new String(db.get("a")) === "2,5") - assert(db.iterator("cf1").isEmpty) - assert(db.iterator("cf2").map(toStr).toSet === Set(("a", "3,4,6"))) - assert(db.iterator().map(toStr).toSet === Set(("a", "2,5"))) + assert(db.get("a") === null) + assert(db.iterator().isEmpty) } } } @@ -1486,8 +1116,6 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared assert(metrics.nativeOpsMetrics("totalBytesWrittenByCompaction") >=0) assert(metrics.nativeOpsMetrics("totalBytesWrittenByFlush") >= 0) - assert(metrics.numExternalColFamilies > 0) - assert(metrics.numInternalColFamilies >= 0) } withTempDir { dir => @@ -2224,7 +1852,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared } } } - } */ + } private def sqlConf = SQLConf.get.clone() @@ -2245,7 +1873,8 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared conf = conf, localRootDir = localDir, hadoopConf = hadoopConf, - loggingId = s"[Thread-${Thread.currentThread.getId}]" + loggingId = s"[Thread-${Thread.currentThread.getId}]", + useColumnFamilies = useColumnFamilies ) db.load(version) func(db) @@ -2327,5 +1956,3 @@ object RocksDBSuite { } } } - -// scalastyle:on diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala index 79214dc78cbd3..a7667723cd0ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala @@ -553,7 +553,8 @@ class TransformWithStateSuite extends StateStoreMetricsTest checkAnswer(df, Seq(("a", "1"), ("b", "1")).toDF()) } - test("transformWithState - test deleteIfExists operator") { + // TODO SPARK-48796 after restart state id will not be the same + ignore("transformWithState - test deleteIfExists operator") { withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, SQLConf.SHUFFLE_PARTITIONS.key -> @@ -573,18 +574,16 @@ class TransformWithStateSuite extends StateStoreMetricsTest TimeMode.None(), OutputMode.Update()) - /* testStream(stream1, OutputMode.Update())( StartStream(checkpointLocation = dirPath), AddData(inputData, ("a", "str1")), CheckNewAnswer(("a", "1", "")), StopStream - ) */ + ) testStream(stream2, OutputMode.Update())( StartStream(checkpointLocation = dirPath), - AddData(inputData, ("a", "str1")), AddData(inputData, ("a", "str2"), ("b", "str3")), - CheckNewAnswer(("a", ""), ("a", "str1"), + CheckNewAnswer(("a", "str1"), ("b", "")), // should not factor in previous count state Execute { q => assert(q.lastProgress.stateOperators(0).customMetrics.get("numValueStateVars") > 0) From bfc003acd50f3eea832773dc579d3e88952c11f1 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 14:25:47 -0700 Subject: [PATCH 14/23] fix scala style after rebase --- .../apache/spark/sql/execution/streaming/state/RocksDB.scala | 3 +-- .../execution/streaming/state/RocksDBStateStoreProvider.scala | 2 +- 2 files changed, 2 insertions(+), 3 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 2635c11e30cb8..e5a0e174d5648 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 @@ -23,8 +23,7 @@ import java.util.concurrent.TimeUnit import javax.annotation.concurrent.GuardedBy import scala.collection.{mutable, Map} -import scala.collection.mutable.{ArrayBuffer, ListBuffer} -import scala.jdk.CollectionConverters._ +import scala.collection.mutable.ListBuffer import scala.ref.WeakReference import scala.util.Try 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 378dc506877e6..a210bc264c795 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 @@ -439,7 +439,7 @@ private[sql] class RocksDBStateStoreProvider private def verify(condition: => Boolean, msg: String): Unit = { if (!condition) { throw new IllegalStateException(msg) } } - + /** * Get the state store of endVersion by applying delta files on the snapshot of snapshotVersion. * If snapshot for snapshotVersion does not exist, an error will be thrown. From 1393a0d85ae35e80c1428faac56de7aa52eda631 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 16:41:40 -0700 Subject: [PATCH 15/23] resolve comments, add comments in code --- .../state/RocksDBStateStoreProvider.scala | 44 ++++++++++++------- 1 file changed, 28 insertions(+), 16 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 a210bc264c795..43caa24051471 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 @@ -72,7 +72,7 @@ private[sql] class RocksDBStateStoreProvider ColumnFamilyUtils.verifyColFamilyOperations("get", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) val value = kvEncoder._2.decodeValue(rocksDB.get(encodedKey)) @@ -105,7 +105,7 @@ private[sql] class RocksDBStateStoreProvider verify(valueEncoder.supportsMultipleValuesPerKey, "valuesIterator requires a encoder " + "that supports multiple values for a single key.") - val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) val encodedValues = rocksDB.get(encodedKey) valueEncoder.decodeValues(encodedValues) } @@ -122,7 +122,7 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) rocksDB.merge(encodedKey, valueEncoder.encodeValue(value)) } @@ -133,7 +133,7 @@ private[sql] class RocksDBStateStoreProvider ColumnFamilyUtils.verifyColFamilyOperations("put", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) rocksDB.put(encodedKey, kvEncoder._2.encodeValue(value)) } @@ -143,7 +143,7 @@ private[sql] class RocksDBStateStoreProvider ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyToIdMap.get(colFamilyName))) + val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) rocksDB.remove(encodedKey) } @@ -155,8 +155,11 @@ private[sql] class RocksDBStateStoreProvider val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() + // As Virtual Column Family attach 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) { - val cfId: Short = colFamilyToIdMap.get(colFamilyName) + val cfId: Short = colFamilyNameToIdMap.get(colFamilyName) rocksDB.prefixScan(ColumnFamilyUtils.getVcfIdBytes(cfId)).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), kvEncoder._2.decodeValue(kv.value)) @@ -191,7 +194,7 @@ private[sql] class RocksDBStateStoreProvider val rowPair = new UnsafeRowPair() val prefix = - kvEncoder._1.encodePrefixKey(prefixKey, Option(colFamilyToIdMap.get(colFamilyName))) + kvEncoder._1.encodePrefixKey(prefixKey, Option(colFamilyNameToIdMap.get(colFamilyName))) rocksDB.prefixScan(prefix).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key, useColumnFamilies), kvEncoder._2.decodeValue(kv.value)) @@ -248,12 +251,12 @@ private[sql] class RocksDBStateStoreProvider // Used for metrics reporting around internal/external column families def internalColFamilyCnt(): Long = { - colFamilyToIdMap.keys.asScala.toSeq + colFamilyNameToIdMap.keys.asScala.toSeq .filter(ColumnFamilyUtils.checkInternalColumnFamilies(_)).size } def externalColFamilyCnt(): Long = { - colFamilyToIdMap.keys.asScala.toSeq + colFamilyNameToIdMap.keys.asScala.toSeq .filter(!ColumnFamilyUtils.checkInternalColumnFamilies(_)).size } @@ -315,7 +318,7 @@ private[sql] class RocksDBStateStoreProvider if (colFamilyExists) { ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) val idPrefix = ColumnFamilyUtils.getVcfIdBytes( - colFamilyToIdMap.get(colFamilyName) + colFamilyNameToIdMap.get(colFamilyName) ) rocksDB.prefixScan(idPrefix).foreach { kv => ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) @@ -356,7 +359,7 @@ private[sql] class RocksDBStateStoreProvider RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) if (useColumnFamilies) { // put default column family only if useColumnFamilies are enabled - colFamilyToIdMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, colFamilyId.shortValue()) + colFamilyNameToIdMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, colFamilyId.shortValue()) } rocksDB // lazy initialization @@ -432,7 +435,7 @@ private[sql] class RocksDBStateStoreProvider private val keyValueEncoderMap = new java.util.concurrent.ConcurrentHashMap[String, (RocksDBKeyStateEncoder, RocksDBValueStateEncoder)] - private val colFamilyToIdMap = new java.util.concurrent.ConcurrentHashMap[String, Short] + private val colFamilyNameToIdMap = new java.util.concurrent.ConcurrentHashMap[String, Short] // TODO SPARK-48796 load column family id from state schema when restarting private val colFamilyId = new AtomicInteger(0) @@ -464,10 +467,19 @@ private[sql] class RocksDBStateStoreProvider } } + /** + * Class for column family related utility functions. + * Verification functions for column family names, column family operation validations etc. + */ private object ColumnFamilyUtils { - private val multColFamiliesDisabledStr = "multiple column families disabled in " + + private val multColFamiliesDisabledStr = "multiple column families is disabled in " + "RocksDBStateStoreProvider" + /** + * Function to get Byte Array for the input virtual column family id. + * + * @param id - id of the column family + */ def getVcfIdBytes(id: Short): Array[Byte] = { val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) @@ -549,7 +561,7 @@ private[sql] class RocksDBStateStoreProvider def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean = false): Unit = { verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) if (!checkColFamilyExists(colFamilyName)) { - colFamilyToIdMap.putIfAbsent(colFamilyName, colFamilyId.incrementAndGet().toShort) + colFamilyNameToIdMap.putIfAbsent(colFamilyName, colFamilyId.incrementAndGet().toShort) } } @@ -559,7 +571,7 @@ private[sql] class RocksDBStateStoreProvider def removeColFamilyIfExists(colFamilyName: String): Boolean = { verifyColFamilyCreationOrDeletion("remove_col_family", colFamilyName) if (checkColFamilyExists(colFamilyName)) { - colFamilyToIdMap.remove(colFamilyName) + colFamilyNameToIdMap.remove(colFamilyName) true } else { false @@ -573,7 +585,7 @@ private[sql] class RocksDBStateStoreProvider * @return - true if the column family exists, false otherwise */ def checkColFamilyExists(colFamilyName: String): Boolean = { - colFamilyToIdMap.keys.asScala.toSeq.contains(colFamilyName) + colFamilyNameToIdMap.keys.asScala.toSeq.contains(colFamilyName) } } } From 23f9d41d729f84a11567b51b0b20ae9dc9fd1650 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 17:23:02 -0700 Subject: [PATCH 16/23] move useColumnFamilies into key encoder API --- .../streaming/state/RocksDBStateEncoder.scala | 77 ++++++++----------- .../state/RocksDBStateStoreProvider.scala | 8 +- .../state/RocksDBStateStoreSuite.scala | 6 +- 3 files changed, 39 insertions(+), 52 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 4cd89c31f9f08..d3af739485cf8 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,8 @@ sealed trait RocksDBKeyStateEncoder { def supportPrefixKeyScan: Boolean def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] - def decodeKey(keyBytes: Array[Byte], hasVcfPrefix: Boolean = false): UnsafeRow + def decodeKey(keyBytes: Array[Byte]): UnsafeRow + def offSetForColFamilyPrefix: Int } sealed trait RocksDBValueStateEncoder { @@ -43,17 +44,19 @@ sealed trait RocksDBValueStateEncoder { } object RocksDBStateEncoder { - def getKeyEncoder(keyStateEncoderSpec: KeyStateEncoderSpec): RocksDBKeyStateEncoder = { + def getKeyEncoder( + keyStateEncoderSpec: KeyStateEncoderSpec, + useColumnFamilies: Boolean): RocksDBKeyStateEncoder = { // Return the key state encoder based on the requested type keyStateEncoderSpec match { case NoPrefixKeyStateEncoderSpec(keySchema) => - new NoPrefixKeyStateEncoder(keySchema) + new NoPrefixKeyStateEncoder(keySchema, useColumnFamilies) case PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) => - new PrefixKeyScanStateEncoder(keySchema, numColsPrefixKey) + new PrefixKeyScanStateEncoder(keySchema, numColsPrefixKey, useColumnFamilies) case RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals) => - new RangeKeyScanStateEncoder(keySchema, orderingOrdinals) + new RangeKeyScanStateEncoder(keySchema, orderingOrdinals, useColumnFamilies) case _ => throw new IllegalArgumentException(s"Unsupported key state encoder spec: " + @@ -118,10 +121,14 @@ object RocksDBStateEncoder { */ class PrefixKeyScanStateEncoder( keySchema: StructType, - numColsPrefixKey: Int) extends RocksDBKeyStateEncoder { + numColsPrefixKey: Int, + useColumnFamilies: Boolean = false) extends RocksDBKeyStateEncoder { import RocksDBStateEncoder._ + override def offSetForColFamilyPrefix: Int = + if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { keySchema.zipWithIndex.take(numColsPrefixKey) } @@ -148,15 +155,12 @@ class PrefixKeyScanStateEncoder( private val joinedRowOnKey = new JoinedRow() override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { - val hasVirtualColFamilyPrefix: Boolean = vcfId.isDefined val prefixKeyEncoded = encodeUnsafeRow(extractPrefixKey(row)) val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) - val offSetForColFamilyPrefix = - if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 val encodedBytes = new Array[Byte](prefixKeyEncoded.length + remainingEncoded.length + 4 + offSetForColFamilyPrefix) - if (hasVirtualColFamilyPrefix) { + if (useColumnFamilies) { Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } @@ -175,11 +179,7 @@ class PrefixKeyScanStateEncoder( encodedBytes } - override def decodeKey( - keyBytes: Array[Byte], - hasVcfPrefix: Boolean = false): UnsafeRow = { - val offSetForColFamilyPrefix = - if (hasVcfPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { val prefixKeyEncodedLen = Platform.getInt( keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) @@ -210,15 +210,10 @@ class PrefixKeyScanStateEncoder( } override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { - val hasVirtualColFamilyPrefix = vcfId.isDefined - - val offSetForColFamilyPrefix = - if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 - val prefixKeyEncoded = encodeUnsafeRow(prefixKey) val prefix = new Array[Byte]( prefixKeyEncoded.length + 4 + offSetForColFamilyPrefix) - if (hasVirtualColFamilyPrefix) { + if (useColumnFamilies) { Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -264,10 +259,14 @@ class PrefixKeyScanStateEncoder( */ class RangeKeyScanStateEncoder( keySchema: StructType, - orderingOrdinals: Seq[Int]) extends RocksDBKeyStateEncoder { + orderingOrdinals: Seq[Int], + useColumnFamilies: Boolean = false) extends RocksDBKeyStateEncoder { import RocksDBStateEncoder._ + override def offSetForColFamilyPrefix: Int = + if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + private val rangeScanKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { orderingOrdinals.map { ordinal => val field = keySchema(ordinal) @@ -523,20 +522,16 @@ class RangeKeyScanStateEncoder( } override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { - val hasVirtualColFamilyPrefix: Boolean = vcfId.isDefined // This prefix key has the columns specified by orderingOrdinals val prefixKey = extractPrefixKey(row) val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) - val offSetForColFamilyPrefix = - if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 - val result = if (orderingOrdinals.length < keySchema.length) { val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + remainingEncoded.length + 4 + offSetForColFamilyPrefix) - if (hasVirtualColFamilyPrefix) { + if (useColumnFamilies) { Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } @@ -557,7 +552,7 @@ class RangeKeyScanStateEncoder( // encode the remaining key as it's empty. val encodedBytes = new Array[Byte]( rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) - if (hasVirtualColFamilyPrefix) { + if (useColumnFamilies) { Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } @@ -571,12 +566,7 @@ class RangeKeyScanStateEncoder( result } - override def decodeKey( - keyBytes: Array[Byte], - hasVcfPrefix: Boolean = false): UnsafeRow = { - val offSetForColFamilyPrefix = - if (hasVcfPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 - + override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { val prefixKeyEncodedLen = Platform.getInt( keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) @@ -612,14 +602,10 @@ class RangeKeyScanStateEncoder( } override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { - val hasVirtualColFamilyPrefix: Boolean = vcfId.isDefined - val offSetForColFamilyPrefix = - if (hasVirtualColFamilyPrefix) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 - val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) - if (hasVirtualColFamilyPrefix) { + if (useColumnFamilies) { Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, vcfId.get) } Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, @@ -644,16 +630,19 @@ class RangeKeyScanStateEncoder( * (offset 0 is the version byte of value 0). That is, if the unsafe row has N bytes, * then the generated array byte will be N+1 bytes. */ -class NoPrefixKeyStateEncoder(keySchema: StructType) +class NoPrefixKeyStateEncoder(keySchema: StructType, useColumnFamilies: Boolean = false) extends RocksDBKeyStateEncoder { import RocksDBStateEncoder._ + override def offSetForColFamilyPrefix: Int = + if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + // Reusable objects private val keyRow = new UnsafeRow(keySchema.size) override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { - if (!vcfId.isDefined) { + if (!useColumnFamilies) { encodeUnsafeRow(row) } else { val bytesToEncode = row.getBytes @@ -678,10 +667,8 @@ class NoPrefixKeyStateEncoder(keySchema: StructType) * @note The UnsafeRow returned is reused across calls, and the UnsafeRow just points to * the given byte array. */ - override def decodeKey( - keyBytes: Array[Byte], - hasVcfPrefix: Boolean = false): UnsafeRow = { - if (hasVcfPrefix) { + override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { + if (useColumnFamilies) { if (keyBytes != null) { // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. keyRow.pointTo( 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 43caa24051471..b6beba7d0aa6d 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 @@ -63,7 +63,7 @@ private[sql] class RocksDBStateStoreProvider ColumnFamilyUtils.createColFamilyIfAbsent(colFamilyName, isInternal) keyValueEncoderMap.putIfAbsent(colFamilyName, - (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec), + (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, useColumnFamilies), RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) } @@ -161,7 +161,7 @@ private[sql] class RocksDBStateStoreProvider if (useColumnFamilies) { val cfId: Short = colFamilyNameToIdMap.get(colFamilyName) rocksDB.prefixScan(ColumnFamilyUtils.getVcfIdBytes(cfId)).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key, true), + rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) if (!isValidated && rowPair.value != null && !useColumnFamilies) { StateStoreProvider.validateStateRowFormat( @@ -196,7 +196,7 @@ private[sql] class RocksDBStateStoreProvider val prefix = kvEncoder._1.encodePrefixKey(prefixKey, Option(colFamilyNameToIdMap.get(colFamilyName))) rocksDB.prefixScan(prefix).map { kv => - rowPair.withRows(kvEncoder._1.decodeKey(kv.key, useColumnFamilies), + rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) rowPair } @@ -355,7 +355,7 @@ private[sql] class RocksDBStateStoreProvider } keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, - (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec), + (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, useColumnFamilies), RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) if (useColumnFamilies) { // put default column family only if useColumnFamilies are enabled 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 01865ec8ac488..e928a7fce6758 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 @@ -937,7 +937,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> "create_col_family", - "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + "entity" -> "multiple column families is disabled in RocksDBStateStoreProvider" ), matchPVals = true ) @@ -974,7 +974,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> "create_col_family", - "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + "entity" -> "multiple column families is disabled in RocksDBStateStoreProvider" ), matchPVals = true ) @@ -1192,7 +1192,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid errorClass = "STATE_STORE_UNSUPPORTED_OPERATION", parameters = Map( "operationType" -> operationName, - "entity" -> "multiple column families disabled in RocksDBStateStoreProvider" + "entity" -> "multiple column families is disabled in RocksDBStateStoreProvider" ), matchPVals = true ) From cf7f7d696795410c07019a50776edc5c70e4d4d7 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 17:28:56 -0700 Subject: [PATCH 17/23] empty commit to trigger gh job From f8b39b97c09603c980411b32855971094814f3f4 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 17:43:51 -0700 Subject: [PATCH 18/23] refactor put prefix function --- .../streaming/state/RocksDBStateEncoder.scala | 42 +++++++++++-------- 1 file changed, 25 insertions(+), 17 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 d3af739485cf8..a75b5c1cabaec 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 @@ -111,6 +111,22 @@ object RocksDBStateEncoder { null } } + + /** + * Put column family Id as a prefix to a pre-allocated byte array. + * + * @param encodedBytes - byte array where col family prefix size is pre-allocated + * @param vcfId - virtual column family Id + * @param useColumnFamilies - if column family is enabled + */ + def putColFamilyPrefix( + encodedBytes: Array[Byte], + vcfId: Option[Short], + useColumnFamilies: Boolean): Unit = { + if (useColumnFamilies) { + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) + } + } } /** @@ -118,6 +134,7 @@ object RocksDBStateEncoder { * * @param keySchema - schema of the key to be encoded * @param numColsPrefixKey - number of columns to be used for prefix key + * @param useColumnFamilies - if column family is enabled for this encoder */ class PrefixKeyScanStateEncoder( keySchema: StructType, @@ -160,9 +177,7 @@ class PrefixKeyScanStateEncoder( val encodedBytes = new Array[Byte](prefixKeyEncoded.length + remainingEncoded.length + 4 + offSetForColFamilyPrefix) - if (useColumnFamilies) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) - } + putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, prefixKeyEncoded.length) @@ -213,9 +228,8 @@ class PrefixKeyScanStateEncoder( val prefixKeyEncoded = encodeUnsafeRow(prefixKey) val prefix = new Array[Byte]( prefixKeyEncoded.length + 4 + offSetForColFamilyPrefix) - if (useColumnFamilies) { - Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, vcfId.get) - } + putColFamilyPrefix(prefix, vcfId, useColumnFamilies) + Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, prefixKeyEncoded.length) Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefix, @@ -256,6 +270,7 @@ class PrefixKeyScanStateEncoder( * * @param keySchema - schema of the key to be encoded * @param orderingOrdinals - the ordinals for which the range scan is constructed + * @param useColumnFamilies - if column family is enabled for this encoder */ class RangeKeyScanStateEncoder( keySchema: StructType, @@ -531,10 +546,7 @@ class RangeKeyScanStateEncoder( val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + remainingEncoded.length + 4 + offSetForColFamilyPrefix) - if (useColumnFamilies) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) - } - + putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, @@ -552,10 +564,8 @@ class RangeKeyScanStateEncoder( // encode the remaining key as it's empty. val encodedBytes = new Array[Byte]( rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) - if (useColumnFamilies) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) - } + putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, @@ -605,9 +615,7 @@ class RangeKeyScanStateEncoder( val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) - if (useColumnFamilies) { - Platform.putShort(prefix, Platform.BYTE_ARRAY_OFFSET, vcfId.get) - } + putColFamilyPrefix(prefix, vcfId, useColumnFamilies) Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, @@ -649,7 +657,7 @@ class NoPrefixKeyStateEncoder(keySchema: StructType, useColumnFamilies: Boolean val encodedBytes = new Array[Byte](bytesToEncode.length + STATE_ENCODING_NUM_VERSION_BYTES + VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) + putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES, STATE_ENCODING_VERSION) // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. From 2c383aeedd214c379ddfb60df9fe4cf373651dce Mon Sep 17 00:00:00 2001 From: jingz-db Date: Wed, 3 Jul 2024 18:02:31 -0700 Subject: [PATCH 19/23] few nits --- .../execution/streaming/state/RocksDBStateStoreProvider.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 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 b6beba7d0aa6d..1ae0a6b1faf0c 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 @@ -155,7 +155,7 @@ private[sql] class RocksDBStateStoreProvider val kvEncoder = keyValueEncoderMap.get(colFamilyName) val rowPair = new UnsafeRowPair() - // As Virtual Column Family attach a column family prefix to the key row, + // 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) { @@ -585,7 +585,7 @@ private[sql] class RocksDBStateStoreProvider * @return - true if the column family exists, false otherwise */ def checkColFamilyExists(colFamilyName: String): Boolean = { - colFamilyNameToIdMap.keys.asScala.toSeq.contains(colFamilyName) + colFamilyNameToIdMap.containsKey(colFamilyName) } } } From 985d2e1b98cb36673bf242028fc0c113a79b0755 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Fri, 5 Jul 2024 11:31:59 -0700 Subject: [PATCH 20/23] resolve partial comments, TODO refactor of RocksDBKeyEncoder --- .../execution/streaming/state/RocksDB.scala | 11 +--- .../streaming/state/RocksDBStateEncoder.scala | 58 +++++++++---------- .../state/RocksDBStateStoreProvider.scala | 13 +++-- .../streaming/state/StateStore.scala | 3 +- .../RocksDBStateStoreIntegrationSuite.scala | 7 ++- .../streaming/state/RocksDBSuite.scala | 3 +- 6 files changed, 45 insertions(+), 50 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 e5a0e174d5648..15ef8832ef358 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 @@ -396,8 +396,7 @@ class RocksDB( /** * Get an iterator of all committed and uncommitted key-value pairs. */ - def iterator(): - Iterator[ByteArrayPair] = { + def iterator(): Iterator[ByteArrayPair] = { val iter = db.newIterator() logInfo(log"Getting iterator from version ${MDC(LogKeys.LOADED_VERSION, loadedVersion)}") @@ -446,8 +445,7 @@ class RocksDB( } } - def prefixScan(prefix: Array[Byte]): - Iterator[ByteArrayPair] = { + def prefixScan(prefix: Array[Byte]): Iterator[ByteArrayPair] = { val iter = db.newIterator() iter.seek(prefix) @@ -802,10 +800,7 @@ class RocksDB( acquireLock.notifyAll() } - private def getDBProperty(property: String): Long = { - // get cumulative sum across all available column families - db.getProperty(property).toLong - } + private def getDBProperty(property: String): Long = db.getProperty(property).toLong private def openDB(): Unit = { assert(db == null) 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 a75b5c1cabaec..09c6f1507a2df 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 @@ -33,7 +33,7 @@ sealed trait RocksDBKeyStateEncoder { def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] def decodeKey(keyBytes: Array[Byte]): UnsafeRow - def offSetForColFamilyPrefix: Int + def offsetForColFamilyPrefix: Int } sealed trait RocksDBValueStateEncoder { @@ -143,7 +143,7 @@ class PrefixKeyScanStateEncoder( import RocksDBStateEncoder._ - override def offSetForColFamilyPrefix: Int = + override def offsetForColFamilyPrefix: Int = if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { @@ -176,19 +176,19 @@ class PrefixKeyScanStateEncoder( val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) val encodedBytes = new Array[Byte](prefixKeyEncoded.length + - remainingEncoded.length + 4 + offSetForColFamilyPrefix) + remainingEncoded.length + 4 + offsetForColFamilyPrefix) putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, prefixKeyEncoded.length) Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, prefixKeyEncoded.length) // NOTE: We don't put the length of remainingEncoded as we can calculate later // on deserialization. Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, encodedBytes, - Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix + prefixKeyEncoded.length, + Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix + prefixKeyEncoded.length, remainingEncoded.length) encodedBytes @@ -196,20 +196,20 @@ class PrefixKeyScanStateEncoder( override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { val prefixKeyEncodedLen = Platform.getInt( - keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) + keyBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) Platform.copyMemory(keyBytes, - Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes val remainingKeyEncodedLen = keyBytes.length - 4 - prefixKeyEncodedLen - - offSetForColFamilyPrefix + offsetForColFamilyPrefix val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + - offSetForColFamilyPrefix + prefixKeyEncodedLen, + offsetForColFamilyPrefix + prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) @@ -227,13 +227,13 @@ class PrefixKeyScanStateEncoder( override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { val prefixKeyEncoded = encodeUnsafeRow(prefixKey) val prefix = new Array[Byte]( - prefixKeyEncoded.length + 4 + offSetForColFamilyPrefix) + prefixKeyEncoded.length + 4 + offsetForColFamilyPrefix) putColFamilyPrefix(prefix, vcfId, useColumnFamilies) - Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, prefixKeyEncoded.length) Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefix, - Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, prefixKeyEncoded.length) prefix } @@ -279,7 +279,7 @@ class RangeKeyScanStateEncoder( import RocksDBStateEncoder._ - override def offSetForColFamilyPrefix: Int = + override def offsetForColFamilyPrefix: Int = if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 private val rangeScanKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { @@ -544,32 +544,32 @@ class RangeKeyScanStateEncoder( val result = if (orderingOrdinals.length < keySchema.length) { val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + - remainingEncoded.length + 4 + offSetForColFamilyPrefix) + remainingEncoded.length + 4 + offsetForColFamilyPrefix) putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) // NOTE: We don't put the length of remainingEncoded as we can calculate later // on deserialization. Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 - + rangeScanKeyEncoded.length + offSetForColFamilyPrefix, + + rangeScanKeyEncoded.length + offsetForColFamilyPrefix, remainingEncoded.length) encodedBytes } else { // if the num of ordering cols is same as num of key schema cols, we don't need to // encode the remaining key as it's empty. val encodedBytes = new Array[Byte]( - rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) + rangeScanKeyEncoded.length + 4 + offsetForColFamilyPrefix) putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) encodedBytes } @@ -578,9 +578,9 @@ class RangeKeyScanStateEncoder( override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { val prefixKeyEncodedLen = Platform.getInt( - keyBytes, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix) + keyBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) - Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, + Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) val prefixKeyDecodedForRangeScan = decodeToUnsafeRow(prefixKeyEncoded, @@ -590,11 +590,11 @@ class RangeKeyScanStateEncoder( if (orderingOrdinals.length < keySchema.length) { // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes val remainingKeyEncodedLen = keyBytes.length - 4 - - prefixKeyEncodedLen - offSetForColFamilyPrefix + prefixKeyEncodedLen - offsetForColFamilyPrefix val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) Platform.copyMemory(keyBytes, - Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix + + Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix + prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) @@ -613,13 +613,13 @@ class RangeKeyScanStateEncoder( override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) - val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offSetForColFamilyPrefix) + val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offsetForColFamilyPrefix) putColFamilyPrefix(prefix, vcfId, useColumnFamilies) - Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offSetForColFamilyPrefix, + Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - prefix, Platform.BYTE_ARRAY_OFFSET + 4 + offSetForColFamilyPrefix, rangeScanKeyEncoded.length) + prefix, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) prefix } @@ -643,7 +643,7 @@ class NoPrefixKeyStateEncoder(keySchema: StructType, useColumnFamilies: Boolean import RocksDBStateEncoder._ - override def offSetForColFamilyPrefix: Int = + override def offsetForColFamilyPrefix: Int = if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 // Reusable objects 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 1ae0a6b1faf0c..5cf83cfe9e74d 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 @@ -114,6 +114,7 @@ private[sql] class RocksDBStateStoreProvider colFamilyName: String = StateStore.DEFAULT_COL_FAMILY_NAME): Unit = { verify(state == UPDATING, "Cannot merge after already committed or aborted") ColumnFamilyUtils.verifyColFamilyOperations("merge", colFamilyName) + val kvEncoder = keyValueEncoderMap.get(colFamilyName) val keyEncoder = kvEncoder._1 val valueEncoder = kvEncoder._2 @@ -187,12 +188,12 @@ private[sql] class RocksDBStateStoreProvider override def prefixScan(prefixKey: UnsafeRow, colFamilyName: String): Iterator[UnsafeRowPair] = { ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) + val kvEncoder = keyValueEncoderMap.get(colFamilyName) require(kvEncoder._1.supportPrefixKeyScan, "Prefix scan requires setting prefix key!") val rowPair = new UnsafeRowPair() - val prefix = kvEncoder._1.encodePrefixKey(prefixKey, Option(colFamilyNameToIdMap.get(colFamilyName))) rocksDB.prefixScan(prefix).map { kv => @@ -313,21 +314,21 @@ private[sql] class RocksDBStateStoreProvider /** Remove column family if exists */ override def removeColFamilyIfExists(colFamilyName: String): Boolean = { verify(useColumnFamilies, "Column families are not supported in this store") + val result = { - val colFamilyExists = ColumnFamilyUtils.checkColFamilyExists(colFamilyName) + val colFamilyId = colFamilyNameToIdMap.get(colFamilyName) + val colFamilyExists = ColumnFamilyUtils.removeColFamilyIfExists(colFamilyName) + if (colFamilyExists) { - ColumnFamilyUtils.verifyColFamilyOperations("prefixScan", colFamilyName) val idPrefix = ColumnFamilyUtils.getVcfIdBytes( - colFamilyNameToIdMap.get(colFamilyName) + colFamilyNameToIdMap.get(colFamilyId) ) rocksDB.prefixScan(idPrefix).foreach { kv => - ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) rocksDB.remove(kv.key) } } colFamilyExists } - ColumnFamilyUtils.removeColFamilyIfExists(colFamilyName) keyValueEncoderMap.remove(colFamilyName) 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 16d7a2bd3ec57..76fd36bd726a6 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 @@ -330,8 +330,7 @@ trait StateStoreProvider { * A value not greater than 0 means the operator doesn't activate prefix * key, and the operator should not call prefixScan method in StateStore. * @param useColumnFamilies Whether the underlying state store uses a single or multiple column - * families; by default we'll use virtual column family if this parameter - * is set to true + * families * @param storeConfs Configurations used by the StateStores * @param hadoopConf Hadoop configuration that could be used by StateStore to save state data * @param useMultipleValuesPerKey Whether the underlying state store needs to support multiple 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 384a31f5aa7e6..8fcd6edf1abb7 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 @@ -164,6 +164,8 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest TestWithBothChangelogCheckpointingEnabledAndDisabled) { colFamiliesEnabled => withSQLConf( SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[RocksDBStateStoreProvider].getName, + // Set an unsupported RocksDB format version and the query should fail if it's passed down + // into RocksDB SQLConf.STATE_STORE_ROCKSDB_FORMAT_VERSION.key -> "100") { val inputData = MemoryStream[Int] val query = inputData.toDS().toDF("value") @@ -175,9 +177,8 @@ class RocksDBStateStoreIntegrationSuite extends StreamTest .outputMode("complete") .start() inputData.addData(1, 2) - query.processAllAvailable() - assert(getFormatVersion(query) == 100) - query.stop() + val e = intercept[StreamingQueryException](query.processAllAvailable()) + assert(e.getCause.getCause.getMessage.contains("Unsupported BlockBasedTable format_version")) } } 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 807b1ee1d2f20..e309b3842cd6f 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 @@ -2006,8 +2006,7 @@ class RocksDBSuite extends AlsoTestWithChangelogCheckpointingEnabled with Shared def toStr(kv: ByteArrayPair): (String, String) = (toStr(kv.key), toStr(kv.value)) - def iterator(db: RocksDB): - Iterator[(String, String)] = db.iterator().map(toStr) + def iterator(db: RocksDB): Iterator[(String, String)] = db.iterator().map(toStr) def listFiles(file: File): Seq[File] = { if (!file.exists()) return Seq.empty From f775e90d534114284621857547e17c3775cdfb46 Mon Sep 17 00:00:00 2001 From: jingz-db Date: Fri, 5 Jul 2024 12:51:12 -0700 Subject: [PATCH 21/23] refactored rocksdbkeyencoder - put vcfId into encoder base class --- .../streaming/state/RocksDBStateEncoder.scala | 114 ++++++++++-------- .../state/RocksDBStateStoreProvider.scala | 63 ++++------ 2 files changed, 90 insertions(+), 87 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 09c6f1507a2df..b057829fc9773 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 @@ -30,10 +30,10 @@ import org.apache.spark.unsafe.Platform sealed trait RocksDBKeyStateEncoder { def supportPrefixKeyScan: Boolean - def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] - def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] + def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] + def encodeKey(row: UnsafeRow): Array[Byte] def decodeKey(keyBytes: Array[Byte]): UnsafeRow - def offsetForColFamilyPrefix: Int + def getColumnFamilyIdBytes(): Array[Byte] } sealed trait RocksDBValueStateEncoder { @@ -43,20 +43,56 @@ sealed trait RocksDBValueStateEncoder { def decodeValues(valueBytes: Array[Byte]): Iterator[UnsafeRow] } +abstract class RocksDBKeyStateEncoderBase( + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short] = None) extends RocksDBKeyStateEncoder { + def offsetForColFamilyPrefix: Int = + if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 + + /** + * 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. + * + * @param encodedBytes - byte array where col family prefix size is pre-allocated + * @param useColumnFamilies - if column family is enabled + */ + protected def encodeColumnFamilyPrefix( + encodedBytes: Array[Byte], + useColumnFamilies: Boolean): Unit = { + if (useColumnFamilies) { + Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, virtualColFamilyId.get) + } + } +} + object RocksDBStateEncoder { def getKeyEncoder( keyStateEncoderSpec: KeyStateEncoderSpec, - useColumnFamilies: Boolean): RocksDBKeyStateEncoder = { + useColumnFamilies: Boolean, + virtualColFamilyId: Option[Short] = None): RocksDBKeyStateEncoder = { // Return the key state encoder based on the requested type keyStateEncoderSpec match { case NoPrefixKeyStateEncoderSpec(keySchema) => - new NoPrefixKeyStateEncoder(keySchema, useColumnFamilies) + new NoPrefixKeyStateEncoder(keySchema, useColumnFamilies, virtualColFamilyId) case PrefixKeyScanStateEncoderSpec(keySchema, numColsPrefixKey) => - new PrefixKeyScanStateEncoder(keySchema, numColsPrefixKey, useColumnFamilies) + new PrefixKeyScanStateEncoder(keySchema, numColsPrefixKey, + useColumnFamilies, virtualColFamilyId) case RangeKeyScanStateEncoderSpec(keySchema, orderingOrdinals) => - new RangeKeyScanStateEncoder(keySchema, orderingOrdinals, useColumnFamilies) + new RangeKeyScanStateEncoder(keySchema, orderingOrdinals, + useColumnFamilies, virtualColFamilyId) case _ => throw new IllegalArgumentException(s"Unsupported key state encoder spec: " + @@ -111,22 +147,6 @@ object RocksDBStateEncoder { null } } - - /** - * Put column family Id as a prefix to a pre-allocated byte array. - * - * @param encodedBytes - byte array where col family prefix size is pre-allocated - * @param vcfId - virtual column family Id - * @param useColumnFamilies - if column family is enabled - */ - def putColFamilyPrefix( - encodedBytes: Array[Byte], - vcfId: Option[Short], - useColumnFamilies: Boolean): Unit = { - if (useColumnFamilies) { - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, vcfId.get) - } - } } /** @@ -139,13 +159,12 @@ object RocksDBStateEncoder { class PrefixKeyScanStateEncoder( keySchema: StructType, numColsPrefixKey: Int, - useColumnFamilies: Boolean = false) extends RocksDBKeyStateEncoder { + useColumnFamilies: Boolean = false, + virtualColFamilyId: Option[Short] = None) + extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) { import RocksDBStateEncoder._ - override def offsetForColFamilyPrefix: Int = - if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 - private val prefixKeyFieldsWithIdx: Seq[(StructField, Int)] = { keySchema.zipWithIndex.take(numColsPrefixKey) } @@ -171,13 +190,13 @@ class PrefixKeyScanStateEncoder( // Reusable objects private val joinedRowOnKey = new JoinedRow() - override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + override def encodeKey(row: UnsafeRow): Array[Byte] = { val prefixKeyEncoded = encodeUnsafeRow(extractPrefixKey(row)) val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) val encodedBytes = new Array[Byte](prefixKeyEncoded.length + remainingEncoded.length + 4 + offsetForColFamilyPrefix) - putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) + encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, prefixKeyEncoded.length) @@ -224,11 +243,11 @@ class PrefixKeyScanStateEncoder( prefixKeyProjection(key) } - override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { val prefixKeyEncoded = encodeUnsafeRow(prefixKey) val prefix = new Array[Byte]( prefixKeyEncoded.length + 4 + offsetForColFamilyPrefix) - putColFamilyPrefix(prefix, vcfId, useColumnFamilies) + encodeColumnFamilyPrefix(prefix, useColumnFamilies) Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, prefixKeyEncoded.length) @@ -275,13 +294,12 @@ class PrefixKeyScanStateEncoder( class RangeKeyScanStateEncoder( keySchema: StructType, orderingOrdinals: Seq[Int], - useColumnFamilies: Boolean = false) extends RocksDBKeyStateEncoder { + useColumnFamilies: Boolean = false, + virtualColFamilyId: Option[Short] = None) + extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) { import RocksDBStateEncoder._ - override def offsetForColFamilyPrefix: Int = - if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 - private val rangeScanKeyFieldsWithOrdinal: Seq[(StructField, Int)] = { orderingOrdinals.map { ordinal => val field = keySchema(ordinal) @@ -536,7 +554,7 @@ class RangeKeyScanStateEncoder( writer.getRow() } - override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + override def encodeKey(row: UnsafeRow): Array[Byte] = { // This prefix key has the columns specified by orderingOrdinals val prefixKey = extractPrefixKey(row) val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) @@ -546,7 +564,7 @@ class RangeKeyScanStateEncoder( val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + remainingEncoded.length + 4 + offsetForColFamilyPrefix) - putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) + encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, @@ -565,7 +583,7 @@ class RangeKeyScanStateEncoder( val encodedBytes = new Array[Byte]( rangeScanKeyEncoded.length + 4 + offsetForColFamilyPrefix) - putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) + encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, @@ -611,11 +629,11 @@ class RangeKeyScanStateEncoder( } } - override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offsetForColFamilyPrefix) - putColFamilyPrefix(prefix, vcfId, useColumnFamilies) + encodeColumnFamilyPrefix(prefix, useColumnFamilies) Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, @@ -638,18 +656,18 @@ class RangeKeyScanStateEncoder( * (offset 0 is the version byte of value 0). That is, if the unsafe row has N bytes, * then the generated array byte will be N+1 bytes. */ -class NoPrefixKeyStateEncoder(keySchema: StructType, useColumnFamilies: Boolean = false) - extends RocksDBKeyStateEncoder { +class NoPrefixKeyStateEncoder( + keySchema: StructType, + useColumnFamilies: Boolean = false, + virtualColFamilyId: Option[Short] = None) + extends RocksDBKeyStateEncoderBase(useColumnFamilies, virtualColFamilyId) { import RocksDBStateEncoder._ - override def offsetForColFamilyPrefix: Int = - if (useColumnFamilies) VIRTUAL_COL_FAMILY_PREFIX_BYTES else 0 - // Reusable objects private val keyRow = new UnsafeRow(keySchema.size) - override def encodeKey(row: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + override def encodeKey(row: UnsafeRow): Array[Byte] = { if (!useColumnFamilies) { encodeUnsafeRow(row) } else { @@ -657,7 +675,7 @@ class NoPrefixKeyStateEncoder(keySchema: StructType, useColumnFamilies: Boolean val encodedBytes = new Array[Byte](bytesToEncode.length + STATE_ENCODING_NUM_VERSION_BYTES + VIRTUAL_COL_FAMILY_PREFIX_BYTES) - putColFamilyPrefix(encodedBytes, vcfId, useColumnFamilies) + encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES, STATE_ENCODING_VERSION) // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. @@ -693,7 +711,7 @@ class NoPrefixKeyStateEncoder(keySchema: StructType, useColumnFamilies: Boolean override def supportPrefixKeyScan: Boolean = false - override def encodePrefixKey(prefixKey: UnsafeRow, vcfId: Option[Short]): Array[Byte] = { + override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { throw new IllegalStateException("This encoder doesn't support prefix key!") } } 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 5cf83cfe9e74d..497d489464488 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 @@ -31,7 +31,6 @@ import org.apache.spark.internal.LogKeys._ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.errors.QueryExecutionErrors import org.apache.spark.sql.types.StructType -import org.apache.spark.unsafe.Platform import org.apache.spark.util.Utils private[sql] class RocksDBStateStoreProvider @@ -60,10 +59,10 @@ private[sql] class RocksDBStateStoreProvider keyStateEncoderSpec: KeyStateEncoderSpec, useMultipleValuesPerKey: Boolean = false, isInternal: Boolean = false): Unit = { - ColumnFamilyUtils.createColFamilyIfAbsent(colFamilyName, isInternal) + val newColFamilyId = ColumnFamilyUtils.createColFamilyIfAbsent(colFamilyName, isInternal) keyValueEncoderMap.putIfAbsent(colFamilyName, - (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, useColumnFamilies), + (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, useColumnFamilies, newColFamilyId), RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) } @@ -72,9 +71,8 @@ private[sql] class RocksDBStateStoreProvider ColumnFamilyUtils.verifyColFamilyOperations("get", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) val value = - kvEncoder._2.decodeValue(rocksDB.get(encodedKey)) + kvEncoder._2.decodeValue(rocksDB.get(kvEncoder._1.encodeKey(key))) if (!isValidated && value != null && !useColumnFamilies) { StateStoreProvider.validateStateRowFormat( @@ -105,8 +103,7 @@ private[sql] class RocksDBStateStoreProvider verify(valueEncoder.supportsMultipleValuesPerKey, "valuesIterator requires a encoder " + "that supports multiple values for a single key.") - val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) - val encodedValues = rocksDB.get(encodedKey) + val encodedValues = rocksDB.get(keyEncoder.encodeKey(key)) valueEncoder.decodeValues(encodedValues) } @@ -123,8 +120,7 @@ private[sql] class RocksDBStateStoreProvider verify(key != null, "Key cannot be null") require(value != null, "Cannot merge a null value") - val encodedKey = keyEncoder.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) - rocksDB.merge(encodedKey, valueEncoder.encodeValue(value)) + rocksDB.merge(keyEncoder.encodeKey(key), valueEncoder.encodeValue(value)) } override def put(key: UnsafeRow, value: UnsafeRow, colFamilyName: String): Unit = { @@ -134,8 +130,7 @@ private[sql] class RocksDBStateStoreProvider ColumnFamilyUtils.verifyColFamilyOperations("put", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) - rocksDB.put(encodedKey, kvEncoder._2.encodeValue(value)) + rocksDB.put(kvEncoder._1.encodeKey(key), kvEncoder._2.encodeValue(value)) } override def remove(key: UnsafeRow, colFamilyName: String): Unit = { @@ -144,8 +139,7 @@ private[sql] class RocksDBStateStoreProvider ColumnFamilyUtils.verifyColFamilyOperations("remove", colFamilyName) val kvEncoder = keyValueEncoderMap.get(colFamilyName) - val encodedKey = kvEncoder._1.encodeKey(key, Option(colFamilyNameToIdMap.get(colFamilyName))) - rocksDB.remove(encodedKey) + rocksDB.remove(kvEncoder._1.encodeKey(key)) } override def iterator(colFamilyName: String): Iterator[UnsafeRowPair] = { @@ -160,8 +154,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) { - val cfId: Short = colFamilyNameToIdMap.get(colFamilyName) - rocksDB.prefixScan(ColumnFamilyUtils.getVcfIdBytes(cfId)).map { kv => + rocksDB.prefixScan(kvEncoder._1.getColumnFamilyIdBytes()).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) if (!isValidated && rowPair.value != null && !useColumnFamilies) { @@ -194,8 +187,7 @@ private[sql] class RocksDBStateStoreProvider "Prefix scan requires setting prefix key!") val rowPair = new UnsafeRowPair() - val prefix = - kvEncoder._1.encodePrefixKey(prefixKey, Option(colFamilyNameToIdMap.get(colFamilyName))) + val prefix = kvEncoder._1.encodePrefixKey(prefixKey) rocksDB.prefixScan(prefix).map { kv => rowPair.withRows(kvEncoder._1.decodeKey(kv.key), kvEncoder._2.decodeValue(kv.value)) @@ -316,14 +308,12 @@ private[sql] class RocksDBStateStoreProvider verify(useColumnFamilies, "Column families are not supported in this store") val result = { - val colFamilyId = colFamilyNameToIdMap.get(colFamilyName) val colFamilyExists = ColumnFamilyUtils.removeColFamilyIfExists(colFamilyName) if (colFamilyExists) { - val idPrefix = ColumnFamilyUtils.getVcfIdBytes( - colFamilyNameToIdMap.get(colFamilyId) - ) - rocksDB.prefixScan(idPrefix).foreach { kv => + val colFamilyIdBytes = + keyValueEncoderMap.get(colFamilyName)._1.getColumnFamilyIdBytes() + rocksDB.prefixScan(colFamilyIdBytes).foreach { kv => rocksDB.remove(kv.key) } } @@ -355,13 +345,16 @@ private[sql] class RocksDBStateStoreProvider " enabled in RocksDBStateStore.") } - keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, - (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, useColumnFamilies), - RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) + var defaultColFamilyId: Option[Short] = None if (useColumnFamilies) { // put default column family only if useColumnFamilies are enabled colFamilyNameToIdMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, colFamilyId.shortValue()) + defaultColFamilyId = Option(colFamilyId.shortValue()) } + keyValueEncoderMap.putIfAbsent(StateStore.DEFAULT_COL_FAMILY_NAME, + (RocksDBStateEncoder.getKeyEncoder(keyStateEncoderSpec, + useColumnFamilies, defaultColFamilyId), + RocksDBStateEncoder.getValueEncoder(valueSchema, useMultipleValuesPerKey))) rocksDB // lazy initialization } @@ -476,17 +469,6 @@ private[sql] class RocksDBStateStoreProvider private val multColFamiliesDisabledStr = "multiple column families is disabled in " + "RocksDBStateStoreProvider" - /** - * Function to get Byte Array for the input virtual column family id. - * - * @param id - id of the column family - */ - def getVcfIdBytes(id: Short): Array[Byte] = { - val encodedBytes = new Array[Byte](VIRTUAL_COL_FAMILY_PREFIX_BYTES) - Platform.putShort(encodedBytes, Platform.BYTE_ARRAY_OFFSET, id) - encodedBytes - } - /** * Function to verify invariants for column family based operations * such as get, put, remove etc. @@ -559,11 +541,14 @@ private[sql] class RocksDBStateStoreProvider /** * Create RocksDB column family, if not created already */ - def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean = false): Unit = { + def createColFamilyIfAbsent(colFamilyName: String, isInternal: Boolean = false): + Option[Short] = { verifyColFamilyCreationOrDeletion("create_col_family", colFamilyName, isInternal) if (!checkColFamilyExists(colFamilyName)) { - colFamilyNameToIdMap.putIfAbsent(colFamilyName, colFamilyId.incrementAndGet().toShort) - } + val newColumnFamilyId = colFamilyId.incrementAndGet().toShort + colFamilyNameToIdMap.putIfAbsent(colFamilyName, newColumnFamilyId) + Option(newColumnFamilyId) + } else None } /** From 95e5f99c1c5aa731f202fff24e824302f1067b2a Mon Sep 17 00:00:00 2001 From: jingz-db Date: Fri, 5 Jul 2024 13:29:47 -0700 Subject: [PATCH 22/23] refactor rocksdbkeyencoder --- .../streaming/state/RocksDBStateEncoder.scala | 125 +++++++++--------- 1 file changed, 59 insertions(+), 66 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 b057829fc9773..202fd224ddfdc 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 @@ -64,15 +64,28 @@ abstract class RocksDBKeyStateEncoderBase( /** * Encode and put column family Id as a prefix to a pre-allocated byte array. * - * @param encodedBytes - byte array where col family prefix size is pre-allocated - * @param useColumnFamilies - if column family is enabled + * @param numBytes - size of byte array to be created for storing key row (without + * column family prefix) + * @return Array[Byte] for an array byte to put encoded key bytes + * Int for a starting offset to put the encoded key bytes */ - protected def encodeColumnFamilyPrefix( - encodedBytes: Array[Byte], - useColumnFamilies: Boolean): Unit = { + protected def encodeColumnFamilyPrefix(numBytes: Int): (Array[Byte], Int) = { + 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) + } + + /** + * Get starting offset for decoding an encoded key byte array. + */ + protected def decodeKeyStartOffset: Int = { + if (useColumnFamilies) { + Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES + } else Platform.BYTE_ARRAY_OFFSET } } @@ -194,43 +207,35 @@ class PrefixKeyScanStateEncoder( val prefixKeyEncoded = encodeUnsafeRow(extractPrefixKey(row)) val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) - val encodedBytes = new Array[Byte](prefixKeyEncoded.length + - remainingEncoded.length + 4 + offsetForColFamilyPrefix) - encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) + val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( + prefixKeyEncoded.length + remainingEncoded.length + 4 + ) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, - prefixKeyEncoded.length) + Platform.putInt(encodedBytes, startingOffset, prefixKeyEncoded.length) Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, - prefixKeyEncoded.length) + encodedBytes, startingOffset + 4, prefixKeyEncoded.length) // NOTE: We don't put the length of remainingEncoded as we can calculate later // on deserialization. Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix + prefixKeyEncoded.length, + encodedBytes, startingOffset + 4 + prefixKeyEncoded.length, remainingEncoded.length) encodedBytes } override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { - val prefixKeyEncodedLen = Platform.getInt( - keyBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix) + val prefixKeyEncodedLen = Platform.getInt(keyBytes, decodeKeyStartOffset) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) - Platform.copyMemory(keyBytes, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, - prefixKeyEncoded, - Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) + Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4, + prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) // Here we calculate the remainingKeyEncodedLen leveraging the length of keyBytes val remainingKeyEncodedLen = keyBytes.length - 4 - prefixKeyEncodedLen - offsetForColFamilyPrefix val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) - Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + - offsetForColFamilyPrefix + prefixKeyEncodedLen, - remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - remainingKeyEncodedLen) + Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4 + prefixKeyEncodedLen, + remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) val prefixKeyDecoded = decodeToUnsafeRow(prefixKeyEncoded, numFields = numColsPrefixKey) val remainingKeyDecoded = decodeToUnsafeRow(remainingKeyEncoded, @@ -245,15 +250,13 @@ class PrefixKeyScanStateEncoder( override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { val prefixKeyEncoded = encodeUnsafeRow(prefixKey) - val prefix = new Array[Byte]( - prefixKeyEncoded.length + 4 + offsetForColFamilyPrefix) - encodeColumnFamilyPrefix(prefix, useColumnFamilies) + val (prefix, startingOffset) = encodeColumnFamilyPrefix( + prefixKeyEncoded.length + 4 + ) - Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, - prefixKeyEncoded.length) + Platform.putInt(prefix, startingOffset, prefixKeyEncoded.length) Platform.copyMemory(prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefix, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, - prefixKeyEncoded.length) + startingOffset + 4, prefixKeyEncoded.length) prefix } @@ -561,44 +564,40 @@ class RangeKeyScanStateEncoder( val result = if (orderingOrdinals.length < keySchema.length) { val remainingEncoded = encodeUnsafeRow(remainingKeyProjection(row)) - val encodedBytes = new Array[Byte](rangeScanKeyEncoded.length + - remainingEncoded.length + 4 + offsetForColFamilyPrefix) + val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( + rangeScanKeyEncoded.length + remainingEncoded.length + 4 + ) - encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, + Platform.putInt(encodedBytes, startingOffset, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, - rangeScanKeyEncoded.length) + encodedBytes, startingOffset + 4, rangeScanKeyEncoded.length) // NOTE: We don't put the length of remainingEncoded as we can calculate later // on deserialization. Platform.copyMemory(remainingEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 - + rangeScanKeyEncoded.length + offsetForColFamilyPrefix, + encodedBytes, startingOffset + 4 + rangeScanKeyEncoded.length, remainingEncoded.length) encodedBytes } else { // if the num of ordering cols is same as num of key schema cols, we don't need to // encode the remaining key as it's empty. - val encodedBytes = new Array[Byte]( - rangeScanKeyEncoded.length + 4 + offsetForColFamilyPrefix) + val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( + rangeScanKeyEncoded.length + 4 + ) - encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) - Platform.putInt(encodedBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, + Platform.putInt(encodedBytes, startingOffset, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, - rangeScanKeyEncoded.length) + encodedBytes, startingOffset + 4, rangeScanKeyEncoded.length) encodedBytes } result } override def decodeKey(keyBytes: Array[Byte]): UnsafeRow = { - val prefixKeyEncodedLen = Platform.getInt( - keyBytes, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix) + val prefixKeyEncodedLen = Platform.getInt(keyBytes, decodeKeyStartOffset) val prefixKeyEncoded = new Array[Byte](prefixKeyEncodedLen) - Platform.copyMemory(keyBytes, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, + Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4, prefixKeyEncoded, Platform.BYTE_ARRAY_OFFSET, prefixKeyEncodedLen) val prefixKeyDecodedForRangeScan = decodeToUnsafeRow(prefixKeyEncoded, @@ -611,9 +610,8 @@ class RangeKeyScanStateEncoder( prefixKeyEncodedLen - offsetForColFamilyPrefix val remainingKeyEncoded = new Array[Byte](remainingKeyEncodedLen) - Platform.copyMemory(keyBytes, - Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix + - prefixKeyEncodedLen, remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, + Platform.copyMemory(keyBytes, decodeKeyStartOffset + 4 + prefixKeyEncodedLen, + remainingKeyEncoded, Platform.BYTE_ARRAY_OFFSET, remainingKeyEncodedLen) val remainingKeyDecoded = decodeToUnsafeRow(remainingKeyEncoded, @@ -631,13 +629,11 @@ class RangeKeyScanStateEncoder( override def encodePrefixKey(prefixKey: UnsafeRow): Array[Byte] = { val rangeScanKeyEncoded = encodeUnsafeRow(encodePrefixKeyForRangeScan(prefixKey)) - val prefix = new Array[Byte](rangeScanKeyEncoded.length + 4 + offsetForColFamilyPrefix) + val (prefix, startingOffset) = encodeColumnFamilyPrefix(rangeScanKeyEncoded.length + 4) - encodeColumnFamilyPrefix(prefix, useColumnFamilies) - Platform.putInt(prefix, Platform.BYTE_ARRAY_OFFSET + offsetForColFamilyPrefix, - rangeScanKeyEncoded.length) + Platform.putInt(prefix, startingOffset, rangeScanKeyEncoded.length) Platform.copyMemory(rangeScanKeyEncoded, Platform.BYTE_ARRAY_OFFSET, - prefix, Platform.BYTE_ARRAY_OFFSET + 4 + offsetForColFamilyPrefix, rangeScanKeyEncoded.length) + prefix, startingOffset + 4, rangeScanKeyEncoded.length) prefix } @@ -672,18 +668,16 @@ class NoPrefixKeyStateEncoder( encodeUnsafeRow(row) } else { val bytesToEncode = row.getBytes - val encodedBytes = new Array[Byte](bytesToEncode.length + - STATE_ENCODING_NUM_VERSION_BYTES + VIRTUAL_COL_FAMILY_PREFIX_BYTES) + val (encodedBytes, startingOffset) = encodeColumnFamilyPrefix( + bytesToEncode.length + + STATE_ENCODING_NUM_VERSION_BYTES + ) - encodeColumnFamilyPrefix(encodedBytes, useColumnFamilies) - Platform.putByte(encodedBytes, Platform.BYTE_ARRAY_OFFSET + VIRTUAL_COL_FAMILY_PREFIX_BYTES, - STATE_ENCODING_VERSION) + Platform.putByte(encodedBytes, startingOffset, STATE_ENCODING_VERSION) // Platform.BYTE_ARRAY_OFFSET is the recommended way to memcopy b/w byte arrays. See Platform. Platform.copyMemory( bytesToEncode, Platform.BYTE_ARRAY_OFFSET, - encodedBytes, Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + - VIRTUAL_COL_FAMILY_PREFIX_BYTES, - bytesToEncode.length) + encodedBytes, startingOffset + STATE_ENCODING_NUM_VERSION_BYTES, bytesToEncode.length) encodedBytes } } @@ -699,8 +693,7 @@ class NoPrefixKeyStateEncoder( // Platform.BYTE_ARRAY_OFFSET is the recommended way refer to the 1st offset. See Platform. keyRow.pointTo( keyBytes, - Platform.BYTE_ARRAY_OFFSET + STATE_ENCODING_NUM_VERSION_BYTES + - VIRTUAL_COL_FAMILY_PREFIX_BYTES, + decodeKeyStartOffset + STATE_ENCODING_NUM_VERSION_BYTES, keyBytes.length - STATE_ENCODING_NUM_VERSION_BYTES - VIRTUAL_COL_FAMILY_PREFIX_BYTES) keyRow } else { From 07b8ae2c6b5799f6f1f9302e0267af9be6542abe Mon Sep 17 00:00:00 2001 From: jingz-db Date: Mon, 8 Jul 2024 09:19:41 -0700 Subject: [PATCH 23/23] suite nits --- .../sql/execution/streaming/state/RocksDBStateStoreSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 e928a7fce6758..f6fffc519d8c3 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 @@ -1124,7 +1124,7 @@ class RocksDBStateStoreSuite extends StateStoreSuiteBase[RocksDBStateStoreProvid assert(store.iterator(cfName).toSeq.length == timerTimestamps.length) // prefix scan - if (!keyEncoder.getClass.toString.contains("No")) { + if (!keyEncoder.isInstanceOf[NoPrefixKeyStateEncoderSpec]) { val keyRow = dataToPrefixKeyRow("a") assert(store.prefixScan(keyRow, cfName).toSeq.length == timerTimestamps.filter(_ < 0).length)