liviazhu-db commented on code in PR #50497:
URL: https://github.com/apache/spark/pull/50497#discussion_r2049379593


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -43,14 +43,120 @@ private[sql] class RocksDBStateStoreProvider
   with SupportsFineGrainedReplay {
   import RocksDBStateStoreProvider._
 
-  class RocksDBStateStore(lastVersion: Long) extends StateStore {
+  /**
+   * Implementation of a state store that uses RocksDB as the backing data 
store.
+   *
+   * This store implements a state machine with the following states:
+   * - UPDATING: The store is being updated and has not yet been committed or 
aborted
+   * - COMMITTED: Updates have been successfully committed
+   * - ABORTED: Updates have been aborted
+   *
+   * Operations are validated against the current state to ensure proper usage:
+   * - Get/put/remove/iterator operations are only allowed in UPDATING state
+   * - Commit is only allowed in UPDATING state
+   * - Abort is allowed in UPDATING or ABORTED state
+   * - Metrics retrieval is only allowed in COMMITTED or ABORTED state
+   *
+   * Each store instance is assigned a unique stamp when created, which is 
used to
+   * verify that operations are performed by the owning thread and to prevent
+   * concurrent modifications to the same store.

Review Comment:
   "concurrent modifications to the same store" -> "concurrent modifications to 
the same StateStoreProvider/NativeRocksDB instance" (store is overloaded)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/state/StatePartitionReader.scala:
##########
@@ -126,6 +126,7 @@ abstract class StatePartitionReaderBase(
         stateStoreColFamilySchema.keyStateEncoderSpec.get,
         useMultipleValuesPerKey = useMultipleValuesPerKey,
         isInternal = isInternal)
+      store.abort()

Review Comment:
   Why do we abort here? Can you leave a comment?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -444,17 +576,57 @@ private[sql] class RocksDBStateStoreProvider
 
   override def stateStoreId: StateStoreId = stateStoreId_
 
-  override def getStore(version: Long, uniqueId: Option[String] = None): 
StateStore = {
+  private lazy val stateMachine: RocksDBStateStoreProviderStateMachine =
+    new RocksDBStateStoreProviderStateMachine(stateStoreId, 
RocksDBConf(storeConf))
+
+  /**
+   * 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 {
+
+      // Determine stamp - either use existing or acquire new
+      val stamp = existingStore.map(_.getReadStamp).getOrElse {

Review Comment:
   I think we don't need this anymore (since we don't create a new store if 
existingStore exists)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProviderStateMachine.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming.state
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import javax.annotation.concurrent.GuardedBy
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.errors.QueryExecutionErrors
+
+/**
+ * A state machine that manages the lifecycle of RocksDB state store instances.
+ *
+ * This class enforces proper state transitions and ensures thread-safety for 
accessing
+ * state stores. It prevents concurrent modifications to the same state store 
by using
+ * a stamp-based locking mechanism.
+ *
+ * State Lifecycle:
+ * - RELEASED: The store is not being accessed by any thread
+ * - ACQUIRED: The store is currently being accessed by a thread
+ * - CLOSED: The store has been closed and can no longer be used
+ *
+ * Valid Transitions:
+ * - RELEASED -> ACQUIRED: When a thread acquires the store
+ * - ACQUIRED -> RELEASED: When a thread releases the store
+ * - RELEASED -> CLOSED: When the store is shut down
+ * - ACQUIRED -> MAINTENANCE: Maintenance can be performed on an acquired store
+ * - RELEASED -> MAINTENANCE: Maintenance can be performed on a released store
+ *
+ * Stamps:
+ * Each time a store is acquired, a unique stamp is generated. This stamp must 
be presented
+ * when performing operations on the store and when releasing it. This ensures 
that only
+ * the thread that acquired the store can release it or perform operations on 
it.
+ */
+class RocksDBStateStoreProviderStateMachine(
+    stateStoreId: StateStoreId,
+    rocksDBConf: RocksDBConf) extends Logging {
+
+  private sealed trait STATE
+  private case object RELEASED extends STATE
+  private case object ACQUIRED extends STATE
+  private case object CLOSED extends STATE
+
+  private sealed abstract class TRANSITION(name: String) {
+    override def toString: String = name
+  }
+  private case object LOAD extends TRANSITION("load")
+  private case object RELEASE extends TRANSITION("release")
+  private case object CLOSE extends TRANSITION("close")
+  private case object MAINTENANCE extends TRANSITION("maintenance")
+
+  private val instanceLock = new Object()
+  @GuardedBy("instanceLock")
+  private var state: STATE = RELEASED
+  @GuardedBy("instanceLock")
+  private var acquiredThreadInfo: AcquiredThreadInfo = _
+
+  // Can be read without holding any locks, but should only be updated when
+  // instanceLock is held.
+  // -1 indicates that the store is not locked.
+  private[sql] val currentValidStamp = new AtomicLong(-1L)
+  @GuardedBy("instanceLock")
+  private var lastValidStamp: Long = 0L
+
+  // Instance lock must be held.
+  private def incAndGetStamp: Long = {
+    lastValidStamp += 1
+    currentValidStamp.set(lastValidStamp)
+    lastValidStamp
+  }
+
+  // Instance lock must be held.
+  private def awaitNotLocked(transition: TRANSITION): Unit = {
+    val waitStartTime = System.nanoTime()
+    def timeWaitedMs = {
+      val elapsedNanos = System.nanoTime() - waitStartTime
+      // Convert from nanoseconds to milliseconds
+      TimeUnit.MILLISECONDS.convert(elapsedNanos, TimeUnit.NANOSECONDS)
+    }
+    while (state == ACQUIRED && timeWaitedMs < 
rocksDBConf.lockAcquireTimeoutMs) {
+      instanceLock.wait(10)
+    }
+    if (state == ACQUIRED) {
+      val newAcquiredThreadInfo = AcquiredThreadInfo()
+      val stackTraceOutput = 
acquiredThreadInfo.threadRef.get.get.getStackTrace.mkString("\n")
+      val loggingId = s"StateStoreId(opId=${stateStoreId.operatorId}," +
+        s"partId=${stateStoreId.partitionId},name=${stateStoreId.storeName})"
+      throw QueryExecutionErrors.unreleasedThreadError(loggingId, 
transition.toString,
+        newAcquiredThreadInfo.toString(), acquiredThreadInfo.toString(), 
timeWaitedMs,
+        stackTraceOutput)
+    }
+  }
+
+  /**
+   * Returns oldState, newState.
+   * Throws error if transition is illegal.
+   * MUST be called for every StateStoreProvider method.
+   * Caller MUST hold instance lock.
+   */
+  private def validateAndTransitionState(transition: TRANSITION): (STATE, 
STATE) = {
+    val oldState = state
+    val newState = transition match {
+      case LOAD =>
+        oldState match {
+          case RELEASED => ACQUIRED
+          case ACQUIRED => throw new IllegalStateException("Cannot lock when 
state is LOCKED")

Review Comment:
   Can you create new error classes for these? (as well as the below 
IllegalStateExceptions)



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -43,14 +43,120 @@ private[sql] class RocksDBStateStoreProvider
   with SupportsFineGrainedReplay {
   import RocksDBStateStoreProvider._
 
-  class RocksDBStateStore(lastVersion: Long) extends StateStore {
+  /**
+   * Implementation of a state store that uses RocksDB as the backing data 
store.

Review Comment:
   Could you link the design doc here? Might be useful for future readers to 
see the state machine diagram. Or you can draw it in ASCII art :D 



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -43,14 +43,120 @@ private[sql] class RocksDBStateStoreProvider
   with SupportsFineGrainedReplay {
   import RocksDBStateStoreProvider._
 
-  class RocksDBStateStore(lastVersion: Long) extends StateStore {
+  /**
+   * Implementation of a state store that uses RocksDB as the backing data 
store.
+   *
+   * This store implements a state machine with the following states:
+   * - UPDATING: The store is being updated and has not yet been committed or 
aborted
+   * - COMMITTED: Updates have been successfully committed
+   * - ABORTED: Updates have been aborted
+   *
+   * Operations are validated against the current state to ensure proper usage:
+   * - Get/put/remove/iterator operations are only allowed in UPDATING state
+   * - Commit is only allowed in UPDATING state
+   * - Abort is allowed in UPDATING or ABORTED state
+   * - Metrics retrieval is only allowed in COMMITTED or ABORTED state
+   *
+   * Each store instance is assigned a unique stamp when created, which is 
used to
+   * verify that operations are performed by the owning thread and to prevent
+   * concurrent modifications to the same store.
+   */
+  class RocksDBStateStore(lastVersion: Long, stamp: Long) extends StateStore {
     /** Trait and classes representing the internal state of the store */
     trait STATE
     case object UPDATING extends STATE
     case object COMMITTED extends STATE
     case object ABORTED extends STATE
 
+    private sealed trait TRANSITION
+    private case object UPDATE extends TRANSITION
+    private case object ABORT extends TRANSITION
+    private case object COMMIT extends TRANSITION
+    private case object METRICS extends TRANSITION
+
     @volatile private var state: STATE = UPDATING
+
+    override def getReadStamp: Long = {

Review Comment:
   I think we don't need this anymore.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProviderStateMachine.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming.state
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import javax.annotation.concurrent.GuardedBy
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.errors.QueryExecutionErrors
+
+/**
+ * A state machine that manages the lifecycle of RocksDB state store instances.
+ *
+ * This class enforces proper state transitions and ensures thread-safety for 
accessing
+ * state stores. It prevents concurrent modifications to the same state store 
by using
+ * a stamp-based locking mechanism.
+ *
+ * State Lifecycle:
+ * - RELEASED: The store is not being accessed by any thread
+ * - ACQUIRED: The store is currently being accessed by a thread
+ * - CLOSED: The store has been closed and can no longer be used
+ *
+ * Valid Transitions:
+ * - RELEASED -> ACQUIRED: When a thread acquires the store
+ * - ACQUIRED -> RELEASED: When a thread releases the store
+ * - RELEASED -> CLOSED: When the store is shut down
+ * - ACQUIRED -> MAINTENANCE: Maintenance can be performed on an acquired store
+ * - RELEASED -> MAINTENANCE: Maintenance can be performed on a released store
+ *
+ * Stamps:
+ * Each time a store is acquired, a unique stamp is generated. This stamp must 
be presented
+ * when performing operations on the store and when releasing it. This ensures 
that only
+ * the thread that acquired the store can release it or perform operations on 
it.
+ */
+class RocksDBStateStoreProviderStateMachine(
+    stateStoreId: StateStoreId,
+    rocksDBConf: RocksDBConf) extends Logging {
+
+  private sealed trait STATE
+  private case object RELEASED extends STATE
+  private case object ACQUIRED extends STATE
+  private case object CLOSED extends STATE
+
+  private sealed abstract class TRANSITION(name: String) {
+    override def toString: String = name
+  }
+  private case object LOAD extends TRANSITION("load")
+  private case object RELEASE extends TRANSITION("release")
+  private case object CLOSE extends TRANSITION("close")
+  private case object MAINTENANCE extends TRANSITION("maintenance")
+
+  private val instanceLock = new Object()
+  @GuardedBy("instanceLock")
+  private var state: STATE = RELEASED
+  @GuardedBy("instanceLock")
+  private var acquiredThreadInfo: AcquiredThreadInfo = _
+
+  // Can be read without holding any locks, but should only be updated when
+  // instanceLock is held.
+  // -1 indicates that the store is not locked.
+  private[sql] val currentValidStamp = new AtomicLong(-1L)
+  @GuardedBy("instanceLock")
+  private var lastValidStamp: Long = 0L
+
+  // Instance lock must be held.
+  private def incAndGetStamp: Long = {
+    lastValidStamp += 1
+    currentValidStamp.set(lastValidStamp)
+    lastValidStamp
+  }
+
+  // Instance lock must be held.
+  private def awaitNotLocked(transition: TRANSITION): Unit = {

Review Comment:
   Can you add some comments in this fn?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProviderStateMachine.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming.state
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import javax.annotation.concurrent.GuardedBy
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.errors.QueryExecutionErrors
+
+/**
+ * A state machine that manages the lifecycle of RocksDB state store instances.

Review Comment:
   Like above, can you link the design doc so people can see the state machine 
diagram (or draw in ASCII)?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProviderStateMachine.scala:
##########
@@ -0,0 +1,186 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.streaming.state
+
+import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicLong
+import javax.annotation.concurrent.GuardedBy
+
+import org.apache.spark.internal.Logging
+import org.apache.spark.sql.errors.QueryExecutionErrors
+
+/**
+ * A state machine that manages the lifecycle of RocksDB state store instances.
+ *
+ * This class enforces proper state transitions and ensures thread-safety for 
accessing
+ * state stores. It prevents concurrent modifications to the same state store 
by using
+ * a stamp-based locking mechanism.
+ *
+ * State Lifecycle:
+ * - RELEASED: The store is not being accessed by any thread
+ * - ACQUIRED: The store is currently being accessed by a thread
+ * - CLOSED: The store has been closed and can no longer be used
+ *
+ * Valid Transitions:
+ * - RELEASED -> ACQUIRED: When a thread acquires the store
+ * - ACQUIRED -> RELEASED: When a thread releases the store
+ * - RELEASED -> CLOSED: When the store is shut down
+ * - ACQUIRED -> MAINTENANCE: Maintenance can be performed on an acquired store
+ * - RELEASED -> MAINTENANCE: Maintenance can be performed on a released store
+ *
+ * Stamps:
+ * Each time a store is acquired, a unique stamp is generated. This stamp must 
be presented
+ * when performing operations on the store and when releasing it. This ensures 
that only
+ * the thread that acquired the store can release it or perform operations on 
it.
+ */
+class RocksDBStateStoreProviderStateMachine(
+    stateStoreId: StateStoreId,
+    rocksDBConf: RocksDBConf) extends Logging {
+
+  private sealed trait STATE
+  private case object RELEASED extends STATE
+  private case object ACQUIRED extends STATE
+  private case object CLOSED extends STATE
+
+  private sealed abstract class TRANSITION(name: String) {
+    override def toString: String = name
+  }
+  private case object LOAD extends TRANSITION("load")
+  private case object RELEASE extends TRANSITION("release")
+  private case object CLOSE extends TRANSITION("close")
+  private case object MAINTENANCE extends TRANSITION("maintenance")
+
+  private val instanceLock = new Object()
+  @GuardedBy("instanceLock")
+  private var state: STATE = RELEASED
+  @GuardedBy("instanceLock")
+  private var acquiredThreadInfo: AcquiredThreadInfo = _
+
+  // Can be read without holding any locks, but should only be updated when
+  // instanceLock is held.
+  // -1 indicates that the store is not locked.
+  private[sql] val currentValidStamp = new AtomicLong(-1L)
+  @GuardedBy("instanceLock")
+  private var lastValidStamp: Long = 0L
+
+  // Instance lock must be held.
+  private def incAndGetStamp: Long = {
+    lastValidStamp += 1
+    currentValidStamp.set(lastValidStamp)
+    lastValidStamp
+  }
+
+  // Instance lock must be held.
+  private def awaitNotLocked(transition: TRANSITION): Unit = {
+    val waitStartTime = System.nanoTime()
+    def timeWaitedMs = {
+      val elapsedNanos = System.nanoTime() - waitStartTime
+      // Convert from nanoseconds to milliseconds
+      TimeUnit.MILLISECONDS.convert(elapsedNanos, TimeUnit.NANOSECONDS)
+    }
+    while (state == ACQUIRED && timeWaitedMs < 
rocksDBConf.lockAcquireTimeoutMs) {
+      instanceLock.wait(10)
+    }
+    if (state == ACQUIRED) {
+      val newAcquiredThreadInfo = AcquiredThreadInfo()
+      val stackTraceOutput = 
acquiredThreadInfo.threadRef.get.get.getStackTrace.mkString("\n")
+      val loggingId = s"StateStoreId(opId=${stateStoreId.operatorId}," +
+        s"partId=${stateStoreId.partitionId},name=${stateStoreId.storeName})"
+      throw QueryExecutionErrors.unreleasedThreadError(loggingId, 
transition.toString,
+        newAcquiredThreadInfo.toString(), acquiredThreadInfo.toString(), 
timeWaitedMs,
+        stackTraceOutput)
+    }
+  }
+
+  /**
+   * Returns oldState, newState.
+   * Throws error if transition is illegal.
+   * MUST be called for every StateStoreProvider method.
+   * Caller MUST hold instance lock.
+   */
+  private def validateAndTransitionState(transition: TRANSITION): (STATE, 
STATE) = {
+    val oldState = state
+    val newState = transition match {
+      case LOAD =>
+        oldState match {
+          case RELEASED => ACQUIRED
+          case ACQUIRED => throw new IllegalStateException("Cannot lock when 
state is LOCKED")
+          case CLOSED => throw new IllegalStateException("Cannot lock when 
state is CLOSED")
+        }
+      case RELEASE =>
+        oldState match {
+          case RELEASED => throw new IllegalStateException("Cannot unlock when 
state is UNLOCKED")
+          case ACQUIRED => RELEASED
+          case CLOSED => throw new IllegalStateException("Cannot unlock when 
state is CLOSED")
+        }
+      case CLOSE =>
+        oldState match {
+          case RELEASED => CLOSED
+          case ACQUIRED => throw new IllegalStateException("Cannot closed when 
state is LOCKED")
+          case CLOSED => CLOSED
+        }
+      case MAINTENANCE =>
+        oldState match {
+          case RELEASED => RELEASED
+          case ACQUIRED => ACQUIRED
+          case CLOSED => throw new IllegalStateException("Cannot do 
maintenance when state is" +
+            "CLOSED")
+        }
+    }
+    state = newState
+    if (newState == ACQUIRED) {
+      acquiredThreadInfo = AcquiredThreadInfo()
+    }
+    (oldState, newState)
+  }
+
+  def verifyStamp(stamp: Long): Unit = {

Review Comment:
   Can you add docs for all of the public fns?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala:
##########
@@ -74,6 +74,8 @@ trait ReadStateStore {
   /** Version of the data in this store before committing updates. */
   def version: Long
 
+  def getReadStamp: Long = -1

Review Comment:
   I think we can remove this



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreSuite.scala:
##########
@@ -2044,6 +2066,259 @@ class RocksDBStateStoreSuite extends 
StateStoreSuiteBase[RocksDBStateStoreProvid
     }
   }
 
+  test("state transitions with commit and illegal operations") {

Review Comment:
   Can you make sure we're testing all the executions from 
https://docs.google.com/document/d/1olT4gp_DMSz0VEPAs2pImtZmQNflCnj1N8yFDC0VNUk/edit?tab=t.0#bookmark=id.anfb6utfgtxz?
 



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