You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "ericm-db (via GitHub)" <gi...@apache.org> on 2024/01/26 21:08:52 UTC

[PR] [WIP] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

ericm-db opened a new pull request, #44903:
URL: https://github.com/apache/spark/pull/44903

   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://spark.apache.org/contributing.html
     2. Ensure you have added or run the appropriate tests for your PR: https://spark.apache.org/developer-tools.html
     3. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP][SPARK-XXXX] Your PR title ...'.
     4. Be sure to keep the PR description updated to reflect all changes.
     5. Please write your PR title to summarize what this PR proposes.
     6. If possible, provide a concise example to reproduce the issue for a faster review.
     7. If you want to add a new configuration, please read the guideline first for naming configurations in
        'core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala'.
     8. If you want to add or modify an error type or message, please read the guideline first in
        'common/utils/src/main/resources/error/README.md'.
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   
   
   ### Was this patch authored or co-authored using generative AI tooling?
   <!--
   If generative AI tooling has been used in the process of authoring this patch, please include the
   phrase: 'Generated-by: ' followed by the name of the tool and its version.
   If no, write 'No'.
   Please refer to the [ASF Generative Tooling Guidance](https://www.apache.org/legal/generative-tooling.html) for details.
   -->
   


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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1474483400


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -201,6 +201,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
     override def toString(): String = {
       s"HDFSStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
     }
+
+    override def removeColFamilyIfExists(colFamilyName: String): Unit = {
+      throw new UnsupportedOperationException("Removing with " +

Review Comment:
   nit: Do you plan to add this to error class framework in another PR? Regardless, maybe better to clarify removing with what. Let's not pretend they can get context about error from stack trace.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -59,6 +60,76 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S
   override def close(): Unit = {}
 }
 
+class RunningCountMostRecentStatefulProcessor
+  extends StatefulProcessor[String, (String, String), (String, String, String)]
+    with Logging {

Review Comment:
   nit: indentation for with should be also 2 spaces, same as extends.



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -59,6 +60,76 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S
   override def close(): Unit = {}
 }
 
+class RunningCountMostRecentStatefulProcessor
+  extends StatefulProcessor[String, (String, String), (String, String, String)]
+    with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient private var _mostRecent: ValueState[String] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(
+       handle: StatefulProcessorHandle,

Review Comment:
   nit: indentation seems to be off, by one more space



##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -131,6 +202,41 @@ class TransformWithStateSuite extends StateStoreMetricsTest
       )
     }
   }
+
+  test("transformWithState - test deleteIfExists operator") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key ->
+        TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+      val chkptDir = Utils.createTempDir("streaming.metadata").getCanonicalPath

Review Comment:
   nit: Let's use withTempDir as Spark SQL tests mostly use this instead.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDB.scala:
##########
@@ -264,6 +264,23 @@ class RocksDB(
     }
   }
 
+  /**
+   * Remove RocksDB column family, if exists
+   */
+  def removeColFamilyIfExists(colFamilyName: String): Unit = {
+    if (colFamilyName == StateStore.DEFAULT_COL_FAMILY_NAME) {
+      throw new UnsupportedOperationException("Failed to remove column family with reserved " +

Review Comment:
   nit: Maybe more direct? e.g. Removing default column family is not allowed.



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #44903:
URL: https://github.com/apache/spark/pull/44903#issuecomment-1923816033

   GA failure is only from docker integration test which is unrelated.


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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1475118602


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -62,14 +62,14 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S
 
 class RunningCountMostRecentStatefulProcessor
   extends StatefulProcessor[String, (String, String), (String, String, String)]
-    with Logging {
+  with Logging {
   @transient private var _countState: ValueState[Long] = _
   @transient private var _mostRecent: ValueState[String] = _
   @transient var _processorHandle: StatefulProcessorHandle = _
 
   override def init(
-       handle: StatefulProcessorHandle,
-       outputMode: OutputMode) : Unit = {
+    handle: StatefulProcessorHandle,

Review Comment:
   nit: 4 spaces for method params. https://github.com/databricks/scala-style-guide?tab=readme-ov-file#indent



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1473376469


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -59,6 +60,78 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S
   override def close(): Unit = {}
 }
 
+class RunningCountMostRecentStatefulProcessor
+  extends StatefulProcessor[String, (String, String), (String, String, String)]
+    with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient private var _mostRecent: ValueState[String] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(
+       handle: StatefulProcessorHandle,
+       outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    _countState = _processorHandle.getValueState[String, Long]("countState",
+      Encoders.STRING)
+    _mostRecent = _processorHandle.getValueState[String, String]("mostRecent",
+      Encoders.STRING)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[(String, String)],
+      timerValues: TimerValues): Iterator[(String, String, String)] = {
+    val count = _countState.getOption().getOrElse(0L) + 1
+    val mostRecent = _mostRecent.getOption().getOrElse("")
+
+    var output = List[(String, String, String)]()
+    inputRows.foreach { row =>
+      _mostRecent.update(row._2)
+      _countState.update(count)
+      output = (key, count.toString, mostRecent) :: output
+    }
+    output.iterator
+  }
+
+  override def close(): Unit = {}
+}
+
+class RunningCountMostRecentStatefulProcessorWithDeletion
+  extends RunningCountMostRecentStatefulProcessor
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient private var _mostRecent: ValueState[String] = _
+
+  override def init(
+     handle: StatefulProcessorHandle,
+     outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    _processorHandle.deleteIfExists("countState")
+    _countState = _processorHandle.getValueState[String, Long]("countState",

Review Comment:
   Seems like we are reinitializing and using again ? 



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "ericm-db (via GitHub)" <gi...@apache.org>.
ericm-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1475158643


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -62,14 +62,14 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S
 
 class RunningCountMostRecentStatefulProcessor
   extends StatefulProcessor[String, (String, String), (String, String, String)]
-    with Logging {
+  with Logging {
   @transient private var _countState: ValueState[Long] = _
   @transient private var _mostRecent: ValueState[String] = _
   @transient var _processorHandle: StatefulProcessorHandle = _
 
   override def init(
-       handle: StatefulProcessorHandle,
-       outputMode: OutputMode) : Unit = {
+    handle: StatefulProcessorHandle,

Review Comment:
   fixed



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1475126693


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -201,6 +201,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
     override def toString(): String = {
       s"HDFSStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
     }
+
+    override def removeColFamilyIfExists(colFamilyName: String): Unit = {
+      throw new UnsupportedOperationException("Removing with " +

Review Comment:
   FYI I just merged error class PR. 



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "ericm-db (via GitHub)" <gi...@apache.org>.
ericm-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1475168909


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -201,6 +201,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
     override def toString(): String = {
       s"HDFSStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
     }
+
+    override def removeColFamilyIfExists(colFamilyName: String): Unit = {
+      throw new UnsupportedOperationException("Removing with " +

Review Comment:
   Just rebased and added the error classes



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1475302336


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreErrors.scala:
##########
@@ -48,6 +58,18 @@ class StateStoreMultipleColumnFamiliesNotSupportedException(stateStoreProvider:
     errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_MULTIPLE_COLUMN_FAMILIES",
     messageParameters = Map("stateStoreProvider" -> stateStoreProvider)
   )
+class StateStoreRemovingColumnFamiliesNotSupportedException(stateStoreProvider: String)
+  extends SparkUnsupportedOperationException(
+    errorClass = "UNSUPPORTED_FEATURE.STATE_STORE_REMOVING_COLUMN_FAMILIES",
+    messageParameters = Map("stateStoreProvider" -> stateStoreProvider)
+  )
+
+  class StateStoreCannotRemoveDefaultColumnFamily(colFamilyName: String)

Review Comment:
   indent seems off for this line ?



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1473355430


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -43,4 +43,9 @@ private[sql] trait StatefulProcessorHandle extends Serializable {
 
   /** Function to return queryInfo for currently running task */
   def getQueryInfo(): QueryInfo
+
+  /** Function to delete and purge state variable if defined previously

Review Comment:
   For multi-line comments - I believe the text needs to start on the next line



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -117,4 +117,15 @@ class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)
   }
 
   override def getQueryInfo(): QueryInfo = currQueryInfo
+
+  /** Function to delete and purge state variable if defined previously

Review Comment:
   same here



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1471790642


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -201,6 +201,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
     override def toString(): String = {
       s"HDFSStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
     }
+
+    override def removeColFamilyIfExists(colFamilyName: String): Unit = {
+      throw new UnsupportedOperationException("Removing with " +

Review Comment:
   Lets use the NERF model now ?



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "ericm-db (via GitHub)" <gi...@apache.org>.
ericm-db commented on PR #44903:
URL: https://github.com/apache/spark/pull/44903#issuecomment-1922470069

   cc @HeartSaVioR, addressed all comments


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


Re: [PR] [WIP] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1468231575


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -201,6 +201,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
     override def toString(): String = {
       s"HDFSStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
     }
+
+    override def removeColFamilyIfExists(colFamilyName: String): Unit = {
+      throw new UnsupportedOperationException("Removing multiple column families with " +

Review Comment:
   Nit: `Removing column family with`



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/RocksDBStateStoreProvider.scala:
##########
@@ -189,6 +189,11 @@ private[sql] class RocksDBStateStoreProvider
 
     /** Return the [[RocksDB]] instance in this store. This is exposed mainly for testing. */
     def dbInstance(): RocksDB = rocksDB
+
+    /** Remove column family if exists */
+     override def removeColFamilyIfExists(colFamilyName: String): Unit = {
+      rocksDB.removeColFamilyIfExists(colFamilyName)

Review Comment:
   nit: indent ?



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


Re: [PR] [WIP] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1468231464


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -117,4 +117,18 @@ class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)
   }
 
   override def getQueryInfo(): QueryInfo = currQueryInfo
+
+  /** Function to delete state info if it exists and return state that was previously there
+   *
+   * @param stateName  - name of the state variable
+   * @param keyEncoder - Spark SQL Encoder for key

Review Comment:
   We don't have these 2 params here ?



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "ericm-db (via GitHub)" <gi...@apache.org>.
ericm-db commented on PR #44903:
URL: https://github.com/apache/spark/pull/44903#issuecomment-1920038156

   cc @HeartSaVioR 


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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1471790142


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -43,4 +43,9 @@ private[sql] trait StatefulProcessorHandle extends Serializable {
 
   /** Function to return queryInfo for currently running task */
   def getQueryInfo(): QueryInfo
+
+  /** Function to delete state info if it exists and return state that was previously there
+   * @param stateName - name of the state variable
+   * */

Review Comment:
   Nit: comment formatting seems off ?



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StatefulProcessorHandleImpl.scala:
##########
@@ -117,4 +117,15 @@ class StatefulProcessorHandleImpl(store: StateStore, runId: UUID)
   }
 
   override def getQueryInfo(): QueryInfo = currQueryInfo
+
+  /** Function to delete state info if it exists and return state that was previously there
+   *
+   * @param stateName  - name of the state variable
+   * */

Review Comment:
   same here ?



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1471789749


##########
sql/api/src/main/scala/org/apache/spark/sql/streaming/StatefulProcessorHandle.scala:
##########
@@ -43,4 +43,9 @@ private[sql] trait StatefulProcessorHandle extends Serializable {
 
   /** Function to return queryInfo for currently running task */
   def getQueryInfo(): QueryInfo
+
+  /** Function to delete state info if it exists and return state that was previously there

Review Comment:
   Nit: lets just say `Function to delete and purge state variable if defined previously` ?



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1471791653


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala:
##########
@@ -103,6 +103,8 @@ trait ReadStateStore {
  * double resource cleanup.
  */
 trait StateStore extends ReadStateStore {
+  def removeColFamilyIfExists(colFamilyName: String): Unit

Review Comment:
   Lets add a function 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-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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR commented on PR #44903:
URL: https://github.com/apache/spark/pull/44903#issuecomment-1923816281

   Thanks! Merging to master.


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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "ericm-db (via GitHub)" <gi...@apache.org>.
ericm-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1473378072


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -59,6 +60,78 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S
   override def close(): Unit = {}
 }
 
+class RunningCountMostRecentStatefulProcessor
+  extends StatefulProcessor[String, (String, String), (String, String, String)]
+    with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient private var _mostRecent: ValueState[String] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(
+       handle: StatefulProcessorHandle,
+       outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    _countState = _processorHandle.getValueState[String, Long]("countState",
+      Encoders.STRING)
+    _mostRecent = _processorHandle.getValueState[String, String]("mostRecent",
+      Encoders.STRING)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[(String, String)],
+      timerValues: TimerValues): Iterator[(String, String, String)] = {
+    val count = _countState.getOption().getOrElse(0L) + 1
+    val mostRecent = _mostRecent.getOption().getOrElse("")
+
+    var output = List[(String, String, String)]()
+    inputRows.foreach { row =>
+      _mostRecent.update(row._2)
+      _countState.update(count)
+      output = (key, count.toString, mostRecent) :: output
+    }
+    output.iterator
+  }
+
+  override def close(): Unit = {}
+}
+
+class RunningCountMostRecentStatefulProcessorWithDeletion
+  extends RunningCountMostRecentStatefulProcessor
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient private var _mostRecent: ValueState[String] = _
+
+  override def init(
+     handle: StatefulProcessorHandle,
+     outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    _processorHandle.deleteIfExists("countState")
+    _countState = _processorHandle.getValueState[String, Long]("countState",

Review Comment:
   Sorry - how so? We're overriding the `init` in the class we extend from, right?



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1473508849


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -59,6 +60,78 @@ class RunningCountStatefulProcessor extends StatefulProcessor[String, String, (S
   override def close(): Unit = {}
 }
 
+class RunningCountMostRecentStatefulProcessor
+  extends StatefulProcessor[String, (String, String), (String, String, String)]
+    with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient private var _mostRecent: ValueState[String] = _
+  @transient var _processorHandle: StatefulProcessorHandle = _
+
+  override def init(
+       handle: StatefulProcessorHandle,
+       outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    _countState = _processorHandle.getValueState[String, Long]("countState",
+      Encoders.STRING)
+    _mostRecent = _processorHandle.getValueState[String, String]("mostRecent",
+      Encoders.STRING)
+  }
+
+  override def handleInputRows(
+      key: String,
+      inputRows: Iterator[(String, String)],
+      timerValues: TimerValues): Iterator[(String, String, String)] = {
+    val count = _countState.getOption().getOrElse(0L) + 1
+    val mostRecent = _mostRecent.getOption().getOrElse("")
+
+    var output = List[(String, String, String)]()
+    inputRows.foreach { row =>
+      _mostRecent.update(row._2)
+      _countState.update(count)
+      output = (key, count.toString, mostRecent) :: output
+    }
+    output.iterator
+  }
+
+  override def close(): Unit = {}
+}
+
+class RunningCountMostRecentStatefulProcessorWithDeletion
+  extends RunningCountMostRecentStatefulProcessor
+  with Logging {
+  @transient private var _countState: ValueState[Long] = _
+  @transient private var _mostRecent: ValueState[String] = _
+
+  override def init(
+     handle: StatefulProcessorHandle,
+     outputMode: OutputMode) : Unit = {
+    _processorHandle = handle
+    assert(handle.getQueryInfo().getBatchId >= 0)
+    _processorHandle.deleteIfExists("countState")
+    _countState = _processorHandle.getValueState[String, Long]("countState",

Review Comment:
   Discussed offline



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


Re: [PR] [WIP] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "anishshri-db (via GitHub)" <gi...@apache.org>.
anishshri-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1470046088


##########
sql/core/src/test/scala/org/apache/spark/sql/streaming/TransformWithStateSuite.scala:
##########
@@ -131,6 +164,40 @@ class TransformWithStateSuite extends StateStoreMetricsTest
       )
     }
   }
+
+  test("transformWithState - test deleteIfExists operator") {
+    withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key ->
+      classOf[RocksDBStateStoreProvider].getName,
+      SQLConf.SHUFFLE_PARTITIONS.key ->
+        TransformWithStateSuiteUtils.NUM_SHUFFLE_PARTITIONS.toString) {
+      val chkptDir = Utils.createTempDir("streaming.metadata").getCanonicalPath
+      val inputData = MemoryStream[String]
+      val stream1 = inputData.toDS()
+        .groupByKey(x => x)
+        .transformWithState(new RunningCountStatefulProcessor(),

Review Comment:
   Probably better to have 2 state variables in the initial stateful processor and then delete 1 of them in the second stateful processor



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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #44903: [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl
URL: https://github.com/apache/spark/pull/44903


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


Re: [PR] [SPARK-46911][SS] Adding deleteIfExists operator to StatefulProcessorHandleImpl [spark]

Posted by "ericm-db (via GitHub)" <gi...@apache.org>.
ericm-db commented on code in PR #44903:
URL: https://github.com/apache/spark/pull/44903#discussion_r1474838218


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala:
##########
@@ -201,6 +201,11 @@ private[sql] class HDFSBackedStateStoreProvider extends StateStoreProvider with
     override def toString(): String = {
       s"HDFSStateStore[id=(op=${id.operatorId},part=${id.partitionId}),dir=$baseDir]"
     }
+
+    override def removeColFamilyIfExists(colFamilyName: String): Unit = {
+      throw new UnsupportedOperationException("Removing with " +

Review Comment:
   Yeah, I wanted to get the NERF PR merged, first, will rebase this PR after that one is merged to keep this change isolated.



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