You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "HeartSaVioR (via GitHub)" <gi...@apache.org> on 2024/02/01 13:47:33 UTC

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

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