You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "anishshri-db (via GitHub)" <gi...@apache.org> on 2023/05/10 18:04:31 UTC

[GitHub] [spark] anishshri-db commented on a diff in pull request #41099: [SPARK-43421][SS] Implement Changelog based Checkpointing for RocksDB State Store Provider

anishshri-db commented on code in PR #41099:
URL: https://github.com/apache/spark/pull/41099#discussion_r1190239700


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala:
##########
@@ -1940,6 +1940,15 @@ object SQLConf {
       // 5 is the default table format version for RocksDB 6.20.3.
       .createWithDefault(5)
 
+  val STATE_STORE_ROCKSDB_CHANGE_CHECKPOINTING_ENABLED =
+    buildConf("spark.sql.streaming.stateStore.rocksdb.enableChangelogCheckpointing")

Review Comment:
   Nit: maybe better to say `changelogCheckpoint.enabled` ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -56,6 +56,15 @@ class RocksDB(
     hadoopConf: Configuration = new Configuration,
     loggingId: String = "") extends Logging {
 
+  case class RocksDBCheckpoint(checkpointDir: File, version: Long, numKeys: Long) {

Review Comment:
   Is case class preferred pattern for something like this ? cc - @HeartSaVioR 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -129,14 +140,15 @@ class RocksDB(
    * Note that this will copy all the necessary file from DFS to local disk as needed,
    * and possibly restart the native RocksDB instance.
    */
-  def load(version: Long): RocksDB = {
+  def load(version: Long, readOnly: Boolean = false): RocksDB = {
     assert(version >= 0)
     acquire()
     logInfo(s"Loading $version")
     try {
       if (loadedVersion != version) {
         closeDB()
-        val metadata = fileManager.loadCheckpointFromDfs(version, workingDir)
+        val latestSnapshotVersion = fileManager.getLatestSnapshotVersion(version)

Review Comment:
   Do we need this if changelog checkpoint is not enabled ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -150,8 +162,27 @@ class RocksDB(
           metadata.numKeys
         }
         numKeysOnWritingVersion = numKeys
-        numKeysOnLoadedVersion = numKeys
-
+        // Replay change log from the last snapshot to the loaded version.
+        // This will be noop if changelog checkpointing is disabled.
+        for (v <- latestSnapshotVersion + 1 to version) {

Review Comment:
   Prefer to embed this in an if condition for easier readability 



-- 
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: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org