micheal-o commented on code in PR #50742:
URL: https://github.com/apache/spark/pull/50742#discussion_r2078913287


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -89,6 +89,8 @@ private[sql] class HDFSBackedStateStoreProvider extends 
StateStoreProvider with
 
     override def abort(): Unit = {}

Review Comment:
   I thought we are removing `abort` api in readstore?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -365,6 +366,18 @@ private[sql] class RocksDBStateStoreProvider
       }
       result
     }
+
+    override def release(): Unit = {
+      if (state != RELEASED) {

Review Comment:
   ditto, this should not be allowed on statestore



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -446,17 +459,49 @@ private[sql] class RocksDBStateStoreProvider
 
   override def stateStoreId: StateStoreId = stateStoreId_
 
-  override def getStore(version: Long, uniqueId: Option[String] = None): 
StateStore = {
+  /**
+   * Creates and returns a state store with the specified parameters.
+   *
+   * @param version The version of the state store to load
+   * @param uniqueId Optional unique identifier for checkpoint
+   * @param readOnly Whether to open the store in read-only mode
+   * @param existingStore Optional existing store to reuse instead of creating 
a new one
+   * @return The loaded state store
+   */
+  private def loadStateStore(
+      version: Long,
+      uniqueId: Option[String],
+      readOnly: Boolean,
+      existingStore: Option[ReadStateStore] = None): StateStore = {
     try {
       if (version < 0) {
         throw QueryExecutionErrors.unexpectedStateStoreVersion(version)
       }
-      rocksDB.load(
-        version,
-        stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) 
uniqueId else None)
-      new RocksDBStateStore(version)
-    }
-    catch {
+      try {
+        // Load RocksDB store
+        rocksDB.load(
+          version,
+          stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) 
uniqueId else None,
+          readOnly = readOnly)
+
+        // Return appropriate store instance
+        existingStore match {
+          // We need to match like this as opposed to case Some(ss: 
RocksDBStateStore)
+          // because of how the tests create the class in StateStoreRDDSuite
+          case Some(stateStore: ReadStateStore) if 
stateStore.isInstanceOf[RocksDBStateStore] =>
+            stateStore.asInstanceOf[StateStore]
+          case Some(other) =>
+            throw new IllegalArgumentException(s"Existing store must be a 
RocksDBStateStore," +
+              s" store is actually ${other.getClass.getSimpleName}")
+          case None =>
+            // Create new store instance for getStore/getReadStore cases
+            new RocksDBStateStore(version)
+        }
+      } catch {
+        case e: Throwable =>

Review Comment:
   why are we doing this?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala:
##########
@@ -109,8 +116,12 @@ package object state {
       val cleanedF = dataRDD.sparkContext.clean(storeReadFn)
       val wrappedF = (store: ReadStateStore, iter: Iterator[T]) => {
         // Clean up the state store.
-        TaskContext.get().addTaskCompletionListener[Unit](_ => {
-          store.abort()
+        val ctxt = TaskContext.get()
+        ctxt.addTaskCompletionListener[Unit](_ => {
+          if (!StateStoreThreadLocalTracker.isUsedForWriteStore) {

Review Comment:
   add comment



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -446,17 +459,49 @@ private[sql] class RocksDBStateStoreProvider
 
   override def stateStoreId: StateStoreId = stateStoreId_
 
-  override def getStore(version: Long, uniqueId: Option[String] = None): 
StateStore = {
+  /**
+   * Creates and returns a state store with the specified parameters.
+   *
+   * @param version The version of the state store to load
+   * @param uniqueId Optional unique identifier for checkpoint
+   * @param readOnly Whether to open the store in read-only mode
+   * @param existingStore Optional existing store to reuse instead of creating 
a new one
+   * @return The loaded state store
+   */
+  private def loadStateStore(
+      version: Long,
+      uniqueId: Option[String],
+      readOnly: Boolean,
+      existingStore: Option[ReadStateStore] = None): StateStore = {
     try {
       if (version < 0) {
         throw QueryExecutionErrors.unexpectedStateStoreVersion(version)
       }
-      rocksDB.load(
-        version,
-        stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) 
uniqueId else None)
-      new RocksDBStateStore(version)
-    }
-    catch {
+      try {
+        // Load RocksDB store
+        rocksDB.load(
+          version,
+          stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) 
uniqueId else None,
+          readOnly = readOnly)
+
+        // Return appropriate store instance
+        existingStore match {
+          // We need to match like this as opposed to case Some(ss: 
RocksDBStateStore)
+          // because of how the tests create the class in StateStoreRDDSuite
+          case Some(stateStore: ReadStateStore) if 
stateStore.isInstanceOf[RocksDBStateStore] =>

Review Comment:
   this check isn't sufficient because it would succeed if I pass in a 
RocksDBStateStore that wasn't created as readonly. Instead we should check if 
the RocksDBStateStore passed in was created as readonly?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -365,6 +366,18 @@ private[sql] class RocksDBStateStoreProvider
       }
       result
     }
+
+    override def release(): Unit = {
+      if (state != RELEASED) {
+        logInfo(log"Releasing ${MDC(VERSION_NUM, version + 1)} " +

Review Comment:
   why is it version + 1 here?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -194,6 +197,10 @@ private[sql] class HDFSBackedStateStoreProvider extends 
StateStoreProvider with
         log"for ${MDC(LogKeys.STATE_STORE_PROVIDER, this)}")
     }
 
+    override def release(): Unit = {

Review Comment:
   release should only be allowed on readstore and shouldn't be allowed on 
statestore. State store should only allow `abort` or `commit`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -446,17 +459,49 @@ private[sql] class RocksDBStateStoreProvider
 
   override def stateStoreId: StateStoreId = stateStoreId_
 
-  override def getStore(version: Long, uniqueId: Option[String] = None): 
StateStore = {
+  /**
+   * Creates and returns a state store with the specified parameters.
+   *
+   * @param version The version of the state store to load
+   * @param uniqueId Optional unique identifier for checkpoint
+   * @param readOnly Whether to open the store in read-only mode
+   * @param existingStore Optional existing store to reuse instead of creating 
a new one
+   * @return The loaded state store
+   */
+  private def loadStateStore(
+      version: Long,
+      uniqueId: Option[String],
+      readOnly: Boolean,
+      existingStore: Option[ReadStateStore] = None): StateStore = {
     try {
       if (version < 0) {
         throw QueryExecutionErrors.unexpectedStateStoreVersion(version)
       }
-      rocksDB.load(
-        version,
-        stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) 
uniqueId else None)
-      new RocksDBStateStore(version)
-    }
-    catch {
+      try {
+        // Load RocksDB store
+        rocksDB.load(
+          version,
+          stateStoreCkptId = if (storeConf.enableStateStoreCheckpointIds) 
uniqueId else None,
+          readOnly = readOnly)
+
+        // Return appropriate store instance
+        existingStore match {
+          // We need to match like this as opposed to case Some(ss: 
RocksDBStateStore)
+          // because of how the tests create the class in StateStoreRDDSuite
+          case Some(stateStore: ReadStateStore) if 
stateStore.isInstanceOf[RocksDBStateStore] =>
+            stateStore.asInstanceOf[StateStore]
+          case Some(other) =>
+            throw new IllegalArgumentException(s"Existing store must be a 
RocksDBStateStore," +

Review Comment:
   Should we throw error earlier if a bad store was passed in before we do load?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/package.scala:
##########
@@ -67,9 +68,15 @@ package object state {
       val cleanedF = dataRDD.sparkContext.clean(storeUpdateFunction)
       val wrappedF = (store: StateStore, iter: Iterator[T]) => {
         // Abort the state store in case of error
-        TaskContext.get().addTaskCompletionListener[Unit](_ => {
+        val ctxt = TaskContext.get()
+        ctxt.addTaskCompletionListener[Unit](_ => {
           if (!store.hasCommitted) store.abort()
         })
+        ctxt.addTaskFailureListener(new TaskFailureListener {

Review Comment:
   why the failure listener?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -89,6 +89,8 @@ private[sql] class HDFSBackedStateStoreProvider extends 
StateStoreProvider with
 
     override def abort(): Unit = {}

Review Comment:
   ok, I think I remember a conversation we had about leaving it for now. But 
with this new change, can you check if there is still any usage of calling 
abort on readstore?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala:
##########
@@ -950,6 +989,29 @@ object StateStore extends Logging {
     storeProvider.getReadStore(version, stateStoreCkptId)
   }
 
+  def getWriteStore(

Review Comment:
   nit: add func comment



-- 
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-unsubscr...@spark.apache.org

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


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

Reply via email to