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

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

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -286,44 +329,56 @@ class RocksDB(
    */
   def commit(): Long = {
     val newVersion = loadedVersion + 1
-    val checkpointDir = createTempDir("checkpoint")
-    var rocksDBBackgroundThreadPaused = false
     try {
-      // Make sure the directory does not exist. Native RocksDB fails if the directory to
-      // checkpoint exists.
-      Utils.deleteRecursively(checkpointDir)
 
       logInfo(s"Flushing updates for $newVersion")
-      val flushTimeMs = timeTakenMs { db.flush(flushOptions) }
-
-      val compactTimeMs = if (conf.compactOnCommit) {
-        logInfo("Compacting")
-        timeTakenMs { db.compactRange() }
-      } else 0
-
-      logInfo("Pausing background work")
-      val pauseTimeMs = timeTakenMs {
-        db.pauseBackgroundWork() // To avoid files being changed while committing
-        rocksDBBackgroundThreadPaused = true
-      }
 
-      logInfo(s"Creating checkpoint for $newVersion in $checkpointDir")
-      val checkpointTimeMs = timeTakenMs {
-        val cp = Checkpoint.create(db)
-        cp.createCheckpoint(checkpointDir.toString)
+      var compactTimeMs = 0L
+      var flushTimeMs = 0L
+      var checkpointTimeMs = 0L
+      if (shouldCreateSnapshot()) {
+        if (conf.compactOnCommit) {
+          logInfo("Compacting")
+          compactTimeMs = timeTakenMs { db.compactRange() }
+        }
+        // Need to flush the change to disk before creating a checkpoint
+        // because rocksdb wal is disabled.
+        flushTimeMs = timeTakenMs { db.flush(flushOptions) }
+        checkpointTimeMs = timeTakenMs {

Review Comment:
   keep the log for `creating checkpoint`?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -129,14 +140,18 @@ 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 loadSnapshotVersion = if (enableChangelogCheckpointing) {

Review Comment:
   do we support case like switching to disable changelog checkpointing (initial stateful query has changelog checkpointing enabled but restart one doesn't)? if not, what kind of checks do we have to prevent users from doing this?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -369,6 +458,9 @@ class RocksDB(
       flushOptions.close()
       dbOptions.close()
       dbLogger.close()
+      synchronized {
+        latestCheckpoint.foreach(_.close())

Review Comment:
   do we still need this? looks like the next `silentDeleteRecursively` can delete all local temp directories?



-- 
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