HeartSaVioR commented on code in PR #47107:
URL: https://github.com/apache/spark/pull/47107#discussion_r1667805849


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateEncoder.scala:
##########
@@ -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 -

Review Comment:
   This can be also calculated based on decodeKeyStartOffset.
   
   decodeKeyStartOffset + 4 + prefixKeyEncodedLen = starting offset for 
remaining key as encoded
   
   The reason we abstract the start offset for both encode and decode is to let 
the subclasses to not deal with column family prefix directly.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala:
##########
@@ -918,6 +918,297 @@ 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 is 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 is 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 =>
+    testWithColumnFamilies(s"validate rocksdb " +
+      s"${keyEncoder.getClass.toString.split('.').last} correctness",
+      TestWithBothChangelogCheckpointingEnabledAndDisabled) { 
colFamiliesEnabled =>
+        tryWithProviderResource(newStoreProvider(keySchema, keyEncoder,
+          colFamiliesEnabled)) { provider =>
+          val store = provider.getStore(0)
+
+          val cfName = if (colFamiliesEnabled) "testColFamily" else "default"
+          if (colFamiliesEnabled) {
+            store.createColFamilyIfAbsent(cfName, keySchema, valueSchema, 
keyEncoder)
+          }
+
+          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")) {

Review Comment:
   nit: Probably clearer to do pattern matching despite it is a couple lines 
longer. Or one-two liner with isInstanceOf.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to