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

[PR] [SS] Add a check for stateful operator change for streaming [spark]

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

   <!--
   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.
   -->
   Currently user will get a misleading error as org.apache.spark.sql.execution.streaming.state.StateSchemaNotCompatible if restarting query in the same checkpoint location and changing their stateful operator. This PR catches such errors and throws a new error with informative message.
   
   After physical planning, before execution phase, we will read from state metadata with the current operator id to fetch operator name of committed batch with the same operator id. If operator name does not match, throws the error.
   
   ### 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.
   -->
   The current error message is misleading to users. We should provide users with message that can guide them to the real root cause of error.
   
   ### 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'.
   -->
   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.
   -->
   Unit tests.
   
   ### 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.
   -->
   No


-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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

   Thanks Jungtaek for your thorough code review!


-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

Posted by "HeartSaVioR (via GitHub)" <gi...@apache.org>.
HeartSaVioR closed pull request #44927: [SPARK-46906][SS] Add a check for stateful operator change for streaming
URL: https://github.com/apache/spark/pull/44927


-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -82,6 +84,39 @@ class IncrementalExecution(
     .map(SQLConf.SHUFFLE_PARTITIONS.valueConverter)
     .getOrElse(sparkSession.sessionState.conf.numShufflePartitions)
 
+  private def stateCheckpointLocationExists(stateCheckpointLocation: Path): Boolean = {
+    val fileManager =
+      CheckpointFileManager.create(stateCheckpointLocation, hadoopConf)
+    fileManager.exists(stateCheckpointLocation)
+  }
+
+  // A map of all (operatorId -> operatorName) in the state metadata
+  private lazy val opMapInMetadata: Map[Long, String] = {
+    var ret = Map.empty[Long, String]
+    if (stateCheckpointLocationExists(new Path(checkpointLocation))) {

Review Comment:
   nit: let's just inline this method if it is referenced only once.



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -387,8 +433,29 @@ class IncrementalExecution(
       rulesToCompose.reduceLeft { (ruleA, ruleB) => ruleA orElse ruleB }
     }
 
+    private def checkOperatorValidWithMetadata(): Unit = {
+     (opMapInMetadata.keySet ++ opMapInPhysicalPlan.keySet).foreach { opId =>
+       val opInMetadata = opMapInMetadata.getOrElse(opId, "not found")
+       val opInCurBatch = opMapInPhysicalPlan.getOrElse(opId, "not found")
+       if (opInMetadata != opInCurBatch) {
+         throw QueryExecutionErrors.statefulOperatorNotMatchInStateMetadataError(
+           opMapInMetadata.values.toSeq,

Review Comment:
   Shall we print out association between opId and opName in error message? It may be uneasy to understand what is mismatching only with opNames.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],
+         OpsInCurBatchSeq: Seq[String],

Review Comment:
   nit: indentation is off, 4 spaces



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1702,6 +1702,18 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
       new NoSuchElementException("State is either not defined or has already been removed")
   }
 
+  def statefulOperatorNotMatchInStateMetadataError(
+      opsInMetadataSeq: Seq[String],

Review Comment:
   Likewise I commented, should have provided information about association between opId and opName. Only opNames does not seem to be sufficient.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {

Review Comment:
   nit: "with debug message" - maybe better to say "with guidance".



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -45,7 +47,6 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     assert(operatorMetadata.operatorInfo == expectedMetadata.operatorInfo &&
       operatorMetadata.stateStoreInfo.sameElements(expectedMetadata.stateStoreInfo))
   }
-

Review Comment:
   nit: revert this change



##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -387,8 +433,29 @@ class IncrementalExecution(
       rulesToCompose.reduceLeft { (ruleA, ruleB) => ruleA orElse ruleB }
     }
 
+    private def checkOperatorValidWithMetadata(): Unit = {

Review Comment:
   Shall we inline all the logic e.g. building opMapInMetadata and opMapInPhysicalPlan to here? I don't see we use these fields other than here. Let's scope fields and methods be narrower whenever possible. 
   
   That said, You don't need to use rule to build opMapInPhysicalPlan. Let's just use foreach to traverse the plan and build opMapInPhysicalPlan.



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3311,6 +3311,13 @@
     ],
     "sqlState" : "42601"
   },
+  "STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA" : {
+    "message" : [
+      "Streaming stateful operator name does not match with the operator in state metadata. This likely to happen when user changes stateful operator of existing streaming query.",

Review Comment:
   nit: when user adds/removes/changes



##########
docs/sql-error-conditions.md:
##########
@@ -2085,6 +2085,13 @@ The checkpoint seems to be only run with older Spark version(s). Run the streami
 
 '`<optionName>`' must be specified.
 
+### STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA
+
+[SQLSTATE: 42K03](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+Streaming stateful operator name does not match with the operator in state metadata. This likely to happen when user changes stateful operator of existing streaming query.

Review Comment:
   nit: when user adds/removes/changes



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],
+         OpsInCurBatchSeq: Seq[String],
+         ex: Throwable): Unit = {
+        checkError(ex.asInstanceOf[SparkRuntimeException],
+          "STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA", "42K03",
+          Map("OpsInMetadataSeq" -> OpsInMetadataSeq.mkString(", "),
+            "OpsInCurBatchSeq" -> OpsInCurBatchSeq.mkString(", "))
+        )
+      }
+
+      // replace dropDuplicates with dropDuplicatesWithinWatermark
+      testStream(stream.withWatermark("eventTime", "10 seconds")
+        .dropDuplicatesWithinWatermark(), Append)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 2),
+        ExpectFailure[SparkRuntimeException] {
+          (t: Throwable) => {
+            checkOpChangeError(Seq("dedupe"), Seq("dedupeWithinWatermark"), t)
+          }
+        }
+      )
+
+      // replace operator

Review Comment:
   I guess it's redundant?



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],
+         OpsInCurBatchSeq: Seq[String],
+         ex: Throwable): Unit = {
+        checkError(ex.asInstanceOf[SparkRuntimeException],
+          "STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA", "42K03",
+          Map("OpsInMetadataSeq" -> OpsInMetadataSeq.mkString(", "),
+            "OpsInCurBatchSeq" -> OpsInCurBatchSeq.mkString(", "))
+        )
+      }
+
+      // replace dropDuplicates with dropDuplicatesWithinWatermark
+      testStream(stream.withWatermark("eventTime", "10 seconds")
+        .dropDuplicatesWithinWatermark(), Append)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 2),
+        ExpectFailure[SparkRuntimeException] {

Review Comment:
   nit: use `{ t =>` to save one indentation & two lines



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],
+         OpsInCurBatchSeq: Seq[String],
+         ex: Throwable): Unit = {
+        checkError(ex.asInstanceOf[SparkRuntimeException],
+          "STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA", "42K03",
+          Map("OpsInMetadataSeq" -> OpsInMetadataSeq.mkString(", "),
+            "OpsInCurBatchSeq" -> OpsInCurBatchSeq.mkString(", "))
+        )
+      }
+
+      // replace dropDuplicates with dropDuplicatesWithinWatermark
+      testStream(stream.withWatermark("eventTime", "10 seconds")
+        .dropDuplicatesWithinWatermark(), Append)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 2),
+        ExpectFailure[SparkRuntimeException] {
+          (t: Throwable) => {
+            checkOpChangeError(Seq("dedupe"), Seq("dedupeWithinWatermark"), t)
+          }
+        }
+      )
+
+      // replace operator
+      testStream(stream.groupBy("value").count(), Update)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        ExpectFailure[SparkRuntimeException] {
+          (t: Throwable) => {
+            checkOpChangeError(Seq("dedupe"), Seq("stateStoreSave"), t)
+          }
+        }
+      )
+
+      // add operator
+      testStream(stream.dropDuplicates()
+        .groupBy("value").count(), Update)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        ExpectFailure[SparkRuntimeException] {
+          (t: Throwable) => {
+            checkOpChangeError(Seq("dedupe"), Seq("stateStoreSave", "dedupe"), t)
+          }
+        }
+      )
+
+      // remove operator

Review Comment:
   Please split down to separate test case if this is fully isolated with other check.
   
   Btw, this actually brings up food for thought. Do we disallow stateful query to be stateless? E.g. could you simply test the removal of stateful operator with checkpointDir rather than spinning up another checkpoint?
   
   It's OK if we have been supporting the case (although undocumented) and we keep supporting the case. If not, we could just test the case via using checkpointDir.



##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],

Review Comment:
   nit: if you are using more than two lines to define the method, param should start at second line of definition. (In other words, all params should appear at the same indentation.)
   
   https://github.com/databricks/scala-style-guide?tab=readme-ov-file#indent
   
   Also, param should start with lowercase.



-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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

   @jingz-db Mind retriggering GA? You can either manually do this in your fork or simply push an empty commit to do this automatically. Thanks!


-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +215,79 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  Seq("Replace", "Add", "Remove").foreach { operation =>
+    test(s"$operation stateful operator will trigger error with guidance") {
+      withTempDir { checkpointDir =>
+        val inputData = MemoryStream[Int]
+        val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+        testStream(stream.dropDuplicates())(
+          StartStream(checkpointLocation = checkpointDir.toString),
+          AddData(inputData, 1),
+          ProcessAllAvailable(),
+          StopStream)
+
+        val (opsInMetadataSeq, opsInCurBatchSeq, restartStream) = operation match {
+          case "Add" =>
+            (
+              Map(0L -> "dedupe"),
+              Map(0L -> "stateStoreSave", 1L -> "dedupe"),
+              stream.dropDuplicates().groupBy("value").count())
+          case "Replace" =>
+            (Map(0L -> "dedupe"), Map(0L -> "stateStoreSave"), stream.groupBy("value").count())
+          case "Remove" =>
+            (Map(0L -> "dedupe"), Map.empty[Long, String], stream)
+        }
+
+        testStream(restartStream, Update)(
+          StartStream(checkpointLocation = checkpointDir.toString),
+          AddData(inputData, 3),
+          ExpectFailure[SparkRuntimeException] { t => {

Review Comment:
   super nit: another {} is unnecessary after `{ t =>`. multiple lines are allowed after `=>`.



-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],

Review Comment:
   Maybe community has to run the formatter at once for the whole codebase. I'm not sure scalafmt can deal with the whole styles though. It is still good to familiarize Scala style guide for Databricks; it doesn't only contain styles automation can handle.



-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

Posted by "chaoqin-li1123 (via GitHub)" <gi...@apache.org>.
chaoqin-li1123 commented on code in PR #44927:
URL: https://github.com/apache/spark/pull/44927#discussion_r1470419428


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3317,6 +3317,12 @@
     ],
     "sqlState" : "XXKST"
   },
+  "STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA" : {
+    "message" : [
+      "Streaming stateful operator name does not match with the operator in state metadata with the same operator id (id: <operatorId>). Stateful Operator name for current batch: <currentOperatorName>; Operator name in the state metadata: <stateMetadataOperatorName>."

Review Comment:
   Can we explain why this occur to the customer? Like "changing stateful operator of existing streaming query 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-46906][SS] Add a check for stateful operator change for streaming [spark]

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

   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-46906][SS] Add a check for stateful operator change for streaming [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],
+         OpsInCurBatchSeq: Seq[String],
+         ex: Throwable): Unit = {
+        checkError(ex.asInstanceOf[SparkRuntimeException],
+          "STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA", "42K03",
+          Map("OpsInMetadataSeq" -> OpsInMetadataSeq.mkString(", "),
+            "OpsInCurBatchSeq" -> OpsInCurBatchSeq.mkString(", "))
+        )
+      }
+
+      // replace dropDuplicates with dropDuplicatesWithinWatermark
+      testStream(stream.withWatermark("eventTime", "10 seconds")
+        .dropDuplicatesWithinWatermark(), Append)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 2),
+        ExpectFailure[SparkRuntimeException] {
+          (t: Throwable) => {
+            checkOpChangeError(Seq("dedupe"), Seq("dedupeWithinWatermark"), t)
+          }
+        }
+      )
+
+      // replace operator
+      testStream(stream.groupBy("value").count(), Update)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        ExpectFailure[SparkRuntimeException] {
+          (t: Throwable) => {
+            checkOpChangeError(Seq("dedupe"), Seq("stateStoreSave"), t)
+          }
+        }
+      )
+
+      // add operator
+      testStream(stream.dropDuplicates()
+        .groupBy("value").count(), Update)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        ExpectFailure[SparkRuntimeException] {
+          (t: Throwable) => {
+            checkOpChangeError(Seq("dedupe"), Seq("stateStoreSave", "dedupe"), t)
+          }
+        }
+      )
+
+      // remove operator

Review Comment:
   > Do we disallow stateful query to be stateless?
   
   We don't allow even before adding the operator check. Streaming will throw error with message as "state path not found".
   
   > E.g. could you simply test the removal of stateful operator with checkpointDir rather than spinning up another checkpoint?
   
   Done. Restarting a stateless query from a stateful query will now trigger error with message as:
   ```bash
   [STREAMING_STATEFUL_OPERATOR_NOT_MATCH_IN_STATE_METADATA] Streaming stateful operator name does not match with the operator in state metadata. This likely to happen when user adds/removes/changes stateful operator of existing streaming query.
   Stateful operators in the metadata: [(OperatorId: 0 -> OperatorName: dedupe)]; Stateful operators in current batch: []. SQLSTATE: 42K03
   
   ```



-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],

Review Comment:
   Do we have any automation tool for checking this other than `./dev/scalafmt`? This command is listed on the [spark developer tool wiki](https://spark.apache.org/developer-tools.html#:~:text=the%20style%20guide.-,Formatting%20code,-To%20format%20Scala), and is actually quite messy - it will touch all existing files other than only formatting my code change.



-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/OperatorStateMetadataSuite.scala:
##########
@@ -215,4 +216,117 @@ class OperatorStateMetadataSuite extends StreamTest with SharedSparkSession {
     checkError(exc, "STDS_REQUIRED_OPTION_UNSPECIFIED", "42601",
       Map("optionName" -> StateSourceOptions.PATH))
   }
+
+  test("Operator metadata path non-existence should not fail query") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val aggregated =
+        inputData.toDF()
+          .groupBy($"value")
+          .agg(count("*"))
+          .as[(Int, Long)]
+
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 1)),
+        StopStream
+      )
+
+      // Delete operator metadata path
+      val metadataPath = new Path(checkpointDir.toString, s"state/0/_metadata/metadata")
+      val fm = CheckpointFileManager.create(new Path(checkpointDir.getCanonicalPath), hadoopConf)
+      fm.delete(metadataPath)
+
+      // Restart the query
+      testStream(aggregated, Complete)(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 3),
+        CheckLastBatch((3, 2)),
+        StopStream
+      )
+    }
+  }
+
+  test("Changing operator - " +
+    "replace, add, remove operators will trigger error with debug message") {
+    withTempDir { checkpointDir =>
+      val inputData = MemoryStream[Int]
+      val stream = inputData.toDF().withColumn("eventTime", timestamp_seconds($"value"))
+
+      testStream(stream.dropDuplicates())(
+        StartStream(checkpointLocation = checkpointDir.toString),
+        AddData(inputData, 1),
+        ProcessAllAvailable(),
+        StopStream
+      )
+
+      def checkOpChangeError(OpsInMetadataSeq: Seq[String],

Review Comment:
   Got it! Thanks Jungtaek!



-- 
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-46906][SS] Add a check for stateful operator change for streaming [spark]

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


##########
sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala:
##########
@@ -184,6 +185,41 @@ class IncrementalExecution(
     }
   }
 
+  /**

Review Comment:
   Thanks for the review Jungtaek! I also like the idea of adding a map.
   > So the check can be also done after executing physical planning rules, maybe at the end of state.apply()
   
   I tried to add the check after `WriteStatefulOperatorMetadataRule` but this will miss detecting for adding an operator after restart (because the additional operator is already written to metadata). So I keep the check before `WriteStatefulOperatorMetadataRule` and will omit the check if metadata is empty.
   It is also worth noting that if we do not perform the check before writing to metadata and fail the query, untruthful info will be written to state metadata.
   



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