You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by "dengziming (via GitHub)" <gi...@apache.org> on 2023/10/23 02:24:49 UTC

[PR] [SPARK-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

dengziming opened a new pull request, #43481:
URL: https://github.com/apache/spark/pull/43481

   ### What changes were proposed in this pull request?
   Assign the name `UNSUPPORTED_DATA_TYPE_FOR_SIZE_FUNCTION` to the legacy error class `_LEGACY_ERROR_TEMP_2156`.
   Assign the name `UNEXPECTED_VALUE_FOR_START_IN_SLICE_FUNCTION` to the legacy error class `_LEGACY_ERROR_TEMP_2157`.
   Assign the name `UNEXPECTED_VALUE_FOR_LENGTH_IN_SLICE_FUNCTION` to the legacy error class `_LEGACY_ERROR_TEMP_2158`.
   
   
   ### Why are the changes needed?
   To assign proper name as a part of activity in SPARK-37935.
   
   
   ### Does this PR introduce _any_ user-facing change?
   Yes, the error message will include the error class name.
   
   
   ### How was this patch tested?
   `UNSUPPORTED_DATA_TYPE_FOR_SIZE_FUNCTION` can't be produced from user code since it will be intercepted during analyze stage, so I added a unit test in `CollectionExpressionsSuite.scala`.
   `UNEXPECTED_VALUE_FOR_START_IN_SLICE_FUNCTION` and `UNEXPECTED_VALUE_FOR_LENGTH_IN_SLICE_FUNCTION` can be produced from user code so I created `CollectionFunctionsSuite.scala` to test them.
   
   ### Was this patch authored or co-authored using generative AI tooling?
   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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

Posted by "MaxGekk (via GitHub)" <gi...@apache.org>.
MaxGekk closed pull request #43481: [SPARK-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8]
URL: https://github.com/apache/spark/pull/43481


-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1368,25 +1368,29 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
   }
 
   def unsupportedOperandTypeForSizeFunctionError(
-      dataType: DataType): SparkUnsupportedOperationException = {
-    new SparkUnsupportedOperationException(
-      errorClass = "_LEGACY_ERROR_TEMP_2156",
-      messageParameters = Map(
-        "dataType" -> dataType.getClass.getCanonicalName))
+      dataType: DataType): Throwable = {
+    SparkException.internalError(
+      s"The size function doesn't support the operand type ${toSQLType(dataType)}")
   }
 
-  def unexpectedValueForStartInFunctionError(prettyName: String): SparkRuntimeException = {
+  def unexpectedValueForStartInFunctionError(
+      prettyName: String, start: Int): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2157",
+      errorClass = "INVALID_PARAMETER_VALUE.START",
       messageParameters = Map(
-        "prettyName" -> prettyName))
+        "parameter" -> "start",

Review Comment:
   Quote it by `toSQLId`



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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

   +1, LGTM. Merging to master.
   Thank you, @dengziming.


-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1368,25 +1368,29 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
   }
 
   def unsupportedOperandTypeForSizeFunctionError(
-      dataType: DataType): SparkUnsupportedOperationException = {
-    new SparkUnsupportedOperationException(
-      errorClass = "_LEGACY_ERROR_TEMP_2156",
-      messageParameters = Map(
-        "dataType" -> dataType.getClass.getCanonicalName))
+      dataType: DataType): Throwable = {
+    SparkException.internalError(
+      s"The size function doesn't support the operand type ${toSQLType(dataType)}")
   }
 
-  def unexpectedValueForStartInFunctionError(prettyName: String): SparkRuntimeException = {
+  def unexpectedValueForStartInFunctionError(
+      prettyName: String, start: Int): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2157",
+      errorClass = "INVALID_PARAMETER_VALUE.START",
       messageParameters = Map(
-        "prettyName" -> prettyName))
+        "parameter" -> "start",
+        "start" -> start.toString,
+        "functionName" -> toSQLId(prettyName)))
   }
 
-  def unexpectedValueForLengthInFunctionError(prettyName: String): SparkRuntimeException = {
+  def unexpectedValueForLengthInFunctionError(
+      prettyName: String, length: Int): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2158",
+      errorClass = "INVALID_PARAMETER_VALUE.LENGTH",
       messageParameters = Map(
-        "prettyName" -> prettyName))
+        "parameter" -> "length",

Review Comment:
   Please, quote it by `toSQLId`



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan commented on code in PR #43481:
URL: https://github.com/apache/spark/pull/43481#discussion_r1380320255


##########
sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala:
##########
@@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
 import org.apache.spark.sql.catalyst.expressions.objects.InitializeJavaBean
 import org.apache.spark.sql.catalyst.rules.RuleIdCollection
 import org.apache.spark.sql.catalyst.util.BadRecordException
+import org.apache.spark.sql.errors.DataTypeErrorsBase

Review Comment:
   nit: We are already in the package `org.apache.spark.sql.errors` and this import is useless.



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala:
##########
@@ -601,9 +616,9 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     checkEvaluation(Slice(a0, Literal(4), Literal(10)), Seq(4, 5, 6))
     checkEvaluation(Slice(a0, Literal(-1), Literal(2)), Seq(6))
     checkExceptionInExpression[RuntimeException](Slice(a0, Literal(1), Literal(-1)),

Review Comment:
   Could you use `checkErrorInExpression` to check the error class and its parameterers.



##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala:
##########
@@ -601,9 +616,9 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     checkEvaluation(Slice(a0, Literal(4), Literal(10)), Seq(4, 5, 6))
     checkEvaluation(Slice(a0, Literal(-1), Literal(2)), Seq(6))
     checkExceptionInExpression[RuntimeException](Slice(a0, Literal(1), Literal(-1)),

Review Comment:
   Could you use `checkErrorInExpression` to check the error class and its parameters.



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala:
##########
@@ -601,9 +616,9 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper
     checkEvaluation(Slice(a0, Literal(4), Literal(10)), Seq(4, 5, 6))
     checkEvaluation(Slice(a0, Literal(-1), Literal(2)), Seq(6))
     checkExceptionInExpression[RuntimeException](Slice(a0, Literal(1), Literal(-1)),
-      "Unexpected value for length")
+      "Expects `length` greater than or equal to 0, but got -1.")
     checkExceptionInExpression[RuntimeException](Slice(a0, Literal(0), Literal(1)),

Review Comment:
   Please, use `checkErrorInExpression`



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1370,21 +1370,21 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
   def unsupportedOperandTypeForSizeFunctionError(
       dataType: DataType): SparkUnsupportedOperationException = {
     new SparkUnsupportedOperationException(
-      errorClass = "_LEGACY_ERROR_TEMP_2156",
+      errorClass = "UNSUPPORTED_DATA_TYPE_FOR_SIZE_FUNCTION",
       messageParameters = Map(
         "dataType" -> dataType.getClass.getCanonicalName))
   }
 
   def unexpectedValueForStartInFunctionError(prettyName: String): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2157",
+      errorClass = "UNEXPECTED_VALUE_FOR_START_IN_SLICE_FUNCTION",
       messageParameters = Map(
         "prettyName" -> prettyName))

Review Comment:
   Good catch, I also replaced `dataType.getClass.getCanonicalName` above with `toSQLType(dataType)`



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1968,6 +1968,11 @@
           "expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal <invalidValue>."
         ]
       },
+      "LENGTH" : {

Review Comment:
   Not sure this name is appropriate, do you have better idea? @MaxGekk 



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1983,6 +1988,11 @@
           "Expects group index between 0 and <groupCount>, but got <groupIndex>."
         ]
       },
+      "START" : {
+        "message" : [
+          "Expects `start` to start at 1 or start from the end if start is negative, but got <start>."

Review Comment:
   Actually, we expect non-zero argument, so, the parameter `<start>` is not needed. Let's rewrite this:
   ```suggestion
             "Expects a positive or a negative value for `start`, but got 0."
   ```



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -1968,6 +1968,11 @@
           "expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal <invalidValue>."
         ]
       },
+      "LENGTH" : {

Review Comment:
   Let's leave `LENGTH`.



-- 
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-45614][SQL] Assign names to error _LEGACY_ERROR_TEMP_215[6,7,8] [spark]

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


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3213,6 +3225,12 @@
     ],
     "sqlState" : "0A000"
   },
+  "UNSUPPORTED_DATA_TYPE_FOR_SIZE_FUNCTION" : {

Review Comment:
   I think we can remove this error class because:
   ```scala
     override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(ArrayType, MapType))
   ```
   and the case must be unreachable:
   ```scala
         case other => throw QueryExecutionErrors.unsupportedOperandTypeForSizeFunctionError(other)
   ```
   @dengziming Please, replace it by an internal error.



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1370,21 +1370,21 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
   def unsupportedOperandTypeForSizeFunctionError(
       dataType: DataType): SparkUnsupportedOperationException = {
     new SparkUnsupportedOperationException(
-      errorClass = "_LEGACY_ERROR_TEMP_2156",
+      errorClass = "UNSUPPORTED_DATA_TYPE_FOR_SIZE_FUNCTION",
       messageParameters = Map(
         "dataType" -> dataType.getClass.getCanonicalName))
   }
 
   def unexpectedValueForStartInFunctionError(prettyName: String): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2157",
+      errorClass = "UNEXPECTED_VALUE_FOR_START_IN_SLICE_FUNCTION",
       messageParameters = Map(
         "prettyName" -> prettyName))

Review Comment:
   Is the pretty name a id, if so, please, wrap it by `toSQLId`



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3029,6 +3029,18 @@
     ],
     "sqlState" : "42846"
   },
+  "UNEXPECTED_VALUE_FOR_LENGTH_IN_SLICE_FUNCTION" : {

Review Comment:
   Could you create a sub-class of the error class `INVALID_PARAMETER_VALUE`, and follow formats of other sub-classes like:
   ```
   expects ..., but got ... .
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala:
##########
@@ -1370,21 +1370,21 @@ private[sql] object QueryExecutionErrors extends QueryErrorsBase with ExecutionE
   def unsupportedOperandTypeForSizeFunctionError(
       dataType: DataType): SparkUnsupportedOperationException = {
     new SparkUnsupportedOperationException(
-      errorClass = "_LEGACY_ERROR_TEMP_2156",
+      errorClass = "UNSUPPORTED_DATA_TYPE_FOR_SIZE_FUNCTION",
       messageParameters = Map(
         "dataType" -> dataType.getClass.getCanonicalName))
   }
 
   def unexpectedValueForStartInFunctionError(prettyName: String): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2157",
+      errorClass = "UNEXPECTED_VALUE_FOR_START_IN_SLICE_FUNCTION",
       messageParameters = Map(
         "prettyName" -> prettyName))
   }
 
   def unexpectedValueForLengthInFunctionError(prettyName: String): SparkRuntimeException = {
     new SparkRuntimeException(
-      errorClass = "_LEGACY_ERROR_TEMP_2158",
+      errorClass = "UNEXPECTED_VALUE_FOR_LENGTH_IN_SLICE_FUNCTION",
       messageParameters = Map(
         "prettyName" -> prettyName))

Review Comment:
   Use `toSQLId`



##########
sql/core/src/test/scala/org/apache/spark/sql/CollectionFunctionsSuite.scala:
##########
@@ -0,0 +1,45 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql
+
+import org.apache.spark.{SparkFunSuite, SparkRuntimeException}
+import org.apache.spark.sql.test.SharedSparkSession
+
+class CollectionFunctionsSuite extends SparkFunSuite with SharedSparkSession {

Review Comment:
   Move the tests to `QueryExecutionErrorsSuite`



##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -3029,6 +3029,18 @@
     ],
     "sqlState" : "42846"
   },
+  "UNEXPECTED_VALUE_FOR_LENGTH_IN_SLICE_FUNCTION" : {
+    "message" : [
+      "Unexpected value for length in function <prettyName>: length must be greater than or equal to 0."
+    ],
+    "sqlState" : "22003"
+  },
+  "UNEXPECTED_VALUE_FOR_START_IN_SLICE_FUNCTION" : {

Review Comment:
   Please, create a sub-class in `INVALID_PARAMETER_VALUE`.



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