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

[PR] [SPARK-45595] Expose SQLSTATE in error message [spark]

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

   <!--
   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
        'core/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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2767,8 +2767,8 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     } catch {
       case e: SparkArithmeticException =>
         throw new ParseException(
-          errorClass = "_LEGACY_ERROR_TEMP_0061",
-          messageParameters = Map("msg" -> e.getMessage),
+          errorClass = e.getErrorClass,
+          messageParameters = e.getMessageParameters.asScala.toMap,

Review Comment:
   cc @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-45595] Expose SQLSTATE in error message [spark]

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

   @srielau About where to place the information. Could we reference other databases or system? e.g. Oracle. 


-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2039,6 +2039,12 @@
     },
     "sqlState" : "42K07"
   },
+  "INVALID_SCHEMA_OR_RELATION_NAME" : {
+    "message" : [
+      "`<name>` is not a valid name for tables/schemas. Valid names only contain alphabet characters, numbers and _."

Review Comment:
   +1



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)
+    val displaySqlState = if (sqlState == null) "" else s" SQLSTATE: $sqlState"

Review Comment:
   I agree. `undefined` is useless for users. We should catch them at our site during testing.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala:
##########
@@ -192,7 +192,7 @@ class UIUtilsSuite extends SparkFunSuite {
 
   // scalastyle:off line.size.limit
   test("SPARK-44367: Extract errorClass from errorMsg with errorMessageCell") {
-    val e1 = "Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1) (10.221.98.22 executor driver): org.apache.spark.SparkArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL(line 1, position 8) ==\nselect a/b from src\n       ^^^\n\n\tat org.apache.spark.sql.errors.QueryExecutionErrors$.divideByZeroError(QueryExecutionErrors.scala:226)\n\tat org.apache.spark.sql.errors.QueryExecutionErrors.divideByZeroError(QueryExecutionErrors.scala)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:54)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvalua
 tor$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)\n\tat org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:328)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)\n\tat org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:141)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:592)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:595)\n\tat java.util.concurr
 ent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:750)\n\nDriver stacktrace:"
+    val e1 = "Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1) (10.221.98.22 executor driver): org.apache.spark.SparkArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL (line 1, position 8) ==\nselect a/b from src\n       ^^^\n\n\tat org.apache.spark.sql.errors.QueryExecutionErrors$.divideByZeroError(QueryExecutionErrors.scala:226)\n\tat org.apache.spark.sql.errors.QueryExecutionErrors.divideByZeroError(QueryExecutionErrors.scala)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:54)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvalu
 ator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)\n\tat org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:328)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)\n\tat org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:141)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:592)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:595)\n\tat java.util.concur
 rent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:750)\n\nDriver stacktrace:"

Review Comment:
   @cloud-fan This line hasn't touched any UI code yet.
   
   @MaxGekk IIRC, I took this error from where`SQLExecution.withNewExecutionId0` called `SparkThrowableHelper.getMessage` for a job failure. 



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala:
##########
@@ -297,7 +297,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle
             |)
             """.stripMargin)
         }.getMessage
-        assert(message.contains("`t:a` is not a valid name for tables/databases. " +
+        assert(message.contains("`t:a` is not a valid name for tables/schemas. " +

Review Comment:
   @srielau Since you are here, could you use `checkError` (and the error class), please.



##########
sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala:
##########
@@ -644,7 +644,8 @@ class CliSuite extends SparkFunSuite {
 
   test("SPARK-37694: delete [jar|file|archive] shall use spark sql processor") {
     runCliWithin(2.minute, errorResponses = Seq("ParseException"))(
-      "delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'.(line 1, pos 7)")
+      "delete jar dummy.jar;" ->
+        "Syntax error at or near 'jar': missing 'FROM'. SQLSTATE: 42601(line 1, pos 7)")

Review Comment:
   Looking at this, should we add a '.' after SQLSTATE like:
   ```suggestion
           "Syntax error at or near 'jar': missing 'FROM'. SQLSTATE: 42601. (line 1, pos 7)")
   ```



##########
core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala:
##########
@@ -90,7 +90,7 @@ class GraphiteSinkSuite extends SparkFunSuite {
     }
     assert(e.getErrorClass === "GRAPHITE_SINK_PROPERTY_MISSING")
     assert(e.getMessage ===
-      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'host' property.")
+      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'host' property. SQLSTATE: KD000")

Review Comment:
   Since you are here, please, use `checkError`.



##########
sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala:
##########
@@ -2570,7 +2570,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
 
     assert(e3.getCause.isInstanceOf[RuntimeException])
     assert(e3.getCause.getMessage.matches(
-      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true!"))
+      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true! SQLSTATE: P0001"))

Review Comment:
   Could you try to use a regexp to match the SQLSTATE?



##########
core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala:
##########
@@ -103,7 +103,7 @@ class GraphiteSinkSuite extends SparkFunSuite {
     }
     assert(e.getErrorClass === "GRAPHITE_SINK_PROPERTY_MISSING")
     assert(e.getMessage ===
-      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'port' property.")
+      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'port' property. SQLSTATE: KD000")

Review Comment:
   The same, try to use `checkError`.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala:
##########
@@ -297,7 +297,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle
             |)
             """.stripMargin)
         }.getMessage
-        assert(message.contains("`t:a` is not a valid name for tables/databases. " +
+        assert(message.contains("`t:a` is not a valid name for tables/schemas. " +

Review Comment:
   Done



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
docs/sql-error-conditions.md:
##########
@@ -1170,6 +1170,12 @@ The input schema `<inputSchema>` is not a valid schema string.
 
 For more details see [INVALID_SCHEMA](sql-error-conditions-invalid-schema-error-class.html)
 
+### INVALID_SCHEMA_OR_RELATION_NAME
+
+[SQLSTATE: 42602](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+``<name>`` is not a valid name for tables/schemas. Valid names only contain alphabet characters, numbers and _.

Review Comment:
   ```suggestion
   `<name>` is not a valid name for tables/schemas. Valid names only contain alphabet characters, numbers and _.
   ```



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala:
##########
@@ -2570,7 +2570,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
 
     assert(e3.getCause.isInstanceOf[RuntimeException])
     assert(e3.getCause.getMessage.matches(
-      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true!"))
+      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true! SQLSTATE: P0001"))

Review Comment:
   I'm thinking that I want to keep it. SQLSTATE is not really supposed to change. And in this case, we expect people to catch them since it's a user-raised exception.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2767,8 +2767,8 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     } catch {
       case e: SparkArithmeticException =>
         throw new ParseException(
-          errorClass = "_LEGACY_ERROR_TEMP_0061",
-          messageParameters = Map("msg" -> e.getMessage),
+          errorClass = e.getErrorClass,
+          messageParameters = e.getMessageParameters.asScala.toMap,

Review Comment:
   It's broken. I have to fix it or I end up with SQLSTATEs INSIDE the core message. That's why I had to fix it.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala:
##########
@@ -719,7 +720,7 @@ class CliSuite extends SparkFunSuite {
       format = ErrorMessageFormat.PRETTY,
       errorMessage =
         """[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.

Review Comment:
   Just wonder why there is no SQLSTATE here, @srielau any ideas why?



##########
core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala:
##########
@@ -192,7 +192,7 @@ class UIUtilsSuite extends SparkFunSuite {
 
   // scalastyle:off line.size.limit
   test("SPARK-44367: Extract errorClass from errorMsg with errorMessageCell") {
-    val e1 = "Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1) (10.221.98.22 executor driver): org.apache.spark.SparkArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL(line 1, position 8) ==\nselect a/b from src\n       ^^^\n\n\tat org.apache.spark.sql.errors.QueryExecutionErrors$.divideByZeroError(QueryExecutionErrors.scala:226)\n\tat org.apache.spark.sql.errors.QueryExecutionErrors.divideByZeroError(QueryExecutionErrors.scala)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:54)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvalua
 tor$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)\n\tat org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:328)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)\n\tat org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:141)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:592)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:595)\n\tat java.util.concurr
 ent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:750)\n\nDriver stacktrace:"
+    val e1 = "Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1) (10.221.98.22 executor driver): org.apache.spark.SparkArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL (line 1, position 8) ==\nselect a/b from src\n       ^^^\n\n\tat org.apache.spark.sql.errors.QueryExecutionErrors$.divideByZeroError(QueryExecutionErrors.scala:226)\n\tat org.apache.spark.sql.errors.QueryExecutionErrors.divideByZeroError(QueryExecutionErrors.scala)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:54)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvalu
 ator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)\n\tat org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:328)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)\n\tat org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:141)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:592)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:595)\n\tat java.util.concur
 rent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:750)\n\nDriver stacktrace:"

Review Comment:
   Why there is no `SQLSTATE` in the error message, any ideas?



-- 
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-45595] Expose SQLSTATE in error message [spark]

Posted by "cloud-fan (via GitHub)" <gi...@apache.org>.
cloud-fan closed pull request #43438: [SPARK-45595] Expose SQLSTATE in error message
URL: https://github.com/apache/spark/pull/43438


-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out:
##########
@@ -427,9 +427,11 @@ select 1.20E-38BD
 -- !query analysis
 org.apache.spark.sql.catalyst.parser.ParseException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_0061",
+  "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION",
+  "sqlState" : "22003",
   "messageParameters" : {
-    "msg" : "[DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION] Decimal precision 40 exceeds max precision 38."

Review Comment:
   We don't nest messages like this. Are you referring to the Exception class?
   All I do is remove the _TEMP ? How can the original be better?



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala:
##########
@@ -644,7 +644,8 @@ class CliSuite extends SparkFunSuite {
 
   test("SPARK-37694: delete [jar|file|archive] shall use spark sql processor") {
     runCliWithin(2.minute, errorResponses = Seq("ParseException"))(
-      "delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'.(line 1, pos 7)")
+      "delete jar dummy.jar;" ->
+        "Syntax error at or near 'jar': missing 'FROM'. SQLSTATE: 42601(line 1, pos 7)")

Review Comment:
   It's not a sentence. But I agree with a space. I need to find where the (line, pos) is coming from. And prefix the space.



##########
sql/hive/src/test/scala/org/apache/spark/sql/hive/MultiDatabaseSuite.scala:
##########
@@ -297,7 +297,7 @@ class MultiDatabaseSuite extends QueryTest with SQLTestUtils with TestHiveSingle
             |)
             """.stripMargin)
         }.getMessage
-        assert(message.contains("`t:a` is not a valid name for tables/databases. " +
+        assert(message.contains("`t:a` is not a valid name for tables/schemas. " +

Review Comment:
   **Hehe** Yes, I had trouble overcoming my inner pig.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala:
##########
@@ -90,7 +90,7 @@ class GraphiteSinkSuite extends SparkFunSuite {
     }
     assert(e.getErrorClass === "GRAPHITE_SINK_PROPERTY_MISSING")
     assert(e.getMessage ===
-      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'host' property.")
+      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'host' property. SQLSTATE: KD000")

Review Comment:
   Done



##########
core/src/test/scala/org/apache/spark/metrics/sink/GraphiteSinkSuite.scala:
##########
@@ -103,7 +103,7 @@ class GraphiteSinkSuite extends SparkFunSuite {
     }
     assert(e.getErrorClass === "GRAPHITE_SINK_PROPERTY_MISSING")
     assert(e.getMessage ===
-      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'port' property.")
+      "[GRAPHITE_SINK_PROPERTY_MISSING] Graphite sink requires 'port' property. SQLSTATE: KD000")

Review Comment:
   Done



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)
+    val displaySqlState = if (sqlState == null) "" else s" SQLSTATE: $sqlState"

Review Comment:
   Not a fan.... I'd have to say: SQLSTATE: undefined and confuse any tools.
   Note that this only happens for LEGACY. Not posting it as we do today seems better to me. 



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
core/src/test/scala/org/apache/spark/ui/UIUtilsSuite.scala:
##########
@@ -192,7 +192,7 @@ class UIUtilsSuite extends SparkFunSuite {
 
   // scalastyle:off line.size.limit
   test("SPARK-44367: Extract errorClass from errorMsg with errorMessageCell") {
-    val e1 = "Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1) (10.221.98.22 executor driver): org.apache.spark.SparkArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL(line 1, position 8) ==\nselect a/b from src\n       ^^^\n\n\tat org.apache.spark.sql.errors.QueryExecutionErrors$.divideByZeroError(QueryExecutionErrors.scala:226)\n\tat org.apache.spark.sql.errors.QueryExecutionErrors.divideByZeroError(QueryExecutionErrors.scala)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:54)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvalua
 tor$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)\n\tat org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:328)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)\n\tat org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:141)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:592)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:595)\n\tat java.util.concurr
 ent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:750)\n\nDriver stacktrace:"
+    val e1 = "Job aborted due to stage failure: Task 0 in stage 1.0 failed 1 times, most recent failure: Lost task 0.0 in stage 1.0 (TID 1) (10.221.98.22 executor driver): org.apache.spark.SparkArithmeticException: [DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set \"spark.sql.ansi.enabled\" to \"false\" to bypass this error.\n== SQL (line 1, position 8) ==\nselect a/b from src\n       ^^^\n\n\tat org.apache.spark.sql.errors.QueryExecutionErrors$.divideByZeroError(QueryExecutionErrors.scala:226)\n\tat org.apache.spark.sql.errors.QueryExecutionErrors.divideByZeroError(QueryExecutionErrors.scala)\n\tat org.apache.spark.sql.catalyst.expressions.GeneratedClass$GeneratedIteratorForCodegenStage1.processNext(generated.java:54)\n\tat org.apache.spark.sql.execution.BufferedRowIterator.hasNext(BufferedRowIterator.java:43)\n\tat org.apache.spark.sql.execution.WholeStageCodegenEvaluatorFactory$WholeStageCodegenPartitionEvalu
 ator$$anon$1.hasNext(WholeStageCodegenEvaluatorFactory.scala:43)\n\tat org.apache.spark.sql.execution.SparkPlan.$anonfun$getByteArrayRdd$1(SparkPlan.scala:388)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2(RDD.scala:890)\n\tat org.apache.spark.rdd.RDD.$anonfun$mapPartitionsInternal$2$adapted(RDD.scala:890)\n\tat org.apache.spark.rdd.MapPartitionsRDD.compute(MapPartitionsRDD.scala:52)\n\tat org.apache.spark.rdd.RDD.computeOrReadCheckpoint(RDD.scala:364)\n\tat org.apache.spark.rdd.RDD.iterator(RDD.scala:328)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:93)\n\tat org.apache.spark.TaskContext.runTaskWithListeners(TaskContext.scala:161)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:141)\n\tat org.apache.spark.executor.Executor$TaskRunner.$anonfun$run$4(Executor.scala:592)\n\tat org.apache.spark.util.Utils$.tryWithSafeFinally(Utils.scala:1474)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:595)\n\tat java.util.concur
 rent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:750)\n\nDriver stacktrace:"

Review Comment:
   cc @yaooqinn . My hunch is that UI calls certain functions to construct error message by itself, which doesn't have the new SQL state handling.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)
+    val displaySqlState = if (sqlState == null) "" else s" SQLSTATE: $sqlState"
     val displayQueryContext = (if (context.isEmpty) "" else "\n") + context
     val prefix = if (errorClass.startsWith("_LEGACY_ERROR_")) "" else s"[$errorClass] "
-    s"$prefix$displayMessage$displayQueryContext"
+    s"$prefix$displayMessage$displaySqlState$displayQueryContext"

Review Comment:
   @srielau Could you investigate other databases or system? I referenced the Oracle error.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)

Review Comment:
   ```suggestion
       val sqlState = getSqlState(errorClass)
   ```



##########
docs/sql-error-conditions.md:
##########
@@ -1170,6 +1170,12 @@ The input schema `<inputSchema>` is not a valid schema string.
 
 For more details see [INVALID_SCHEMA](sql-error-conditions-invalid-schema-error-class.html)
 
+### INVALID_SCHEMA_OR_RELATION_NAME
+
+[SQLSTATE: 42602](sql-error-conditions-sqlstates.html#class-42-syntax-error-or-access-rule-violation)
+
+``<name>`` is not a valid name for tables/schemas. Valid names only contain alphabet characters, numbers and _.

Review Comment:
   ditto



##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)
+    val displaySqlState = if (sqlState == null) "" else s" SQLSTATE: $sqlState"

Review Comment:
   ```suggestion
       val displaySqlState = if (sqlState == null) "undefined" else s" SQLSTATE: $sqlState"
   ```



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2767,8 +2767,8 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     } catch {
       case e: SparkArithmeticException =>
         throw new ParseException(
-          errorClass = "_LEGACY_ERROR_TEMP_0061",
-          messageParameters = Map("msg" -> e.getMessage),
+          errorClass = e.getErrorClass,
+          messageParameters = e.getMessageParameters.asScala.toMap,

Review Comment:
   Why change this?



##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)
+    val displaySqlState = if (sqlState == null) "" else s" SQLSTATE: $sqlState"
     val displayQueryContext = (if (context.isEmpty) "" else "\n") + context
     val prefix = if (errorClass.startsWith("_LEGACY_ERROR_")) "" else s"[$errorClass] "
-    s"$prefix$displayMessage$displayQueryContext"
+    s"$prefix$displayMessage$displaySqlState$displayQueryContext"

Review Comment:
   I think should put `displaySqlState` in front of `displayMessage`



##########
sql/core/src/test/resources/sql-tests/analyzer-results/ansi/literals.sql.out:
##########
@@ -427,9 +427,11 @@ select 1.20E-38BD
 -- !query analysis
 org.apache.spark.sql.catalyst.parser.ParseException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_0061",
+  "errorClass" : "DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION",
+  "sqlState" : "22003",
   "messageParameters" : {
-    "msg" : "[DECIMAL_PRECISION_EXCEEDS_MAX_PRECISION] Decimal precision 40 exceeds max precision 38."

Review Comment:
   Personally, I feel the origin msg is better.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala:
##########
@@ -719,7 +720,7 @@ class CliSuite extends SparkFunSuite {
       format = ErrorMessageFormat.PRETTY,
       errorMessage =
         """[DIVIDE_BY_ZERO] Division by zero. Use `try_divide` to tolerate divisor being 0 and return NULL instead. If necessary set "spark.sql.ansi.enabled" to "false" to bypass this error.

Review Comment:
   This test checks if the response contains the error message, not to make sure they are exactly the same.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)
+    val displaySqlState = if (sqlState == null) "" else s" SQLSTATE: $sqlState"
     val displayQueryContext = (if (context.isEmpty) "" else "\n") + context
     val prefix = if (errorClass.startsWith("_LEGACY_ERROR_")) "" else s"[$errorClass] "
-    s"$prefix$displayMessage$displayQueryContext"
+    s"$prefix$displayMessage$displaySqlState$displayQueryContext"

Review Comment:
   I disagree. SQLSTATE does not help you read the message. It's 5 random alpha-numeric.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala:
##########
@@ -2767,8 +2767,8 @@ class AstBuilder extends DataTypeAstBuilder with SQLConfHelper with Logging {
     } catch {
       case e: SparkArithmeticException =>
         throw new ParseException(
-          errorClass = "_LEGACY_ERROR_TEMP_0061",
-          messageParameters = Map("msg" -> e.getMessage),
+          errorClass = e.getErrorClass,
+          messageParameters = e.getMessageParameters.asScala.toMap,

Review Comment:
   I would prefer to don't mix the things. Let's focus on exposing the SQL state in this 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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala:
##########
@@ -644,7 +644,7 @@ class CliSuite extends SparkFunSuite {
 
   test("SPARK-37694: delete [jar|file|archive] shall use spark sql processor") {
     runCliWithin(2.minute, errorResponses = Seq("ParseException"))(
-      "delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'.(line 1, pos 7)")
+      "delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'. SQLSTATE: 42601(line 1, pos 7)")

Review Comment:
   ```suggestion
         "delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'. " +
         "SQLSTATE: 42601(line 1, pos 7)")
   ```



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala:
##########
@@ -2570,7 +2570,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
 
     assert(e3.getCause.isInstanceOf[RuntimeException])
     assert(e3.getCause.getMessage.matches(
-      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true!"))
+      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true! SQLSTATE: P0001"))

Review Comment:
   Ugh. I'll try



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2039,6 +2039,12 @@
     },
     "sqlState" : "42K07"
   },
+  "INVALID_SCHEMA_OR_RELATION_NAME" : {
+    "message" : [
+      "`<name>` is not a valid name for tables/schemas. Valid names only contain alphabet characters, numbers and _."

Review Comment:
   Please, remove backticks around <name>. We use `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-45595] Expose SQLSTATE in error message [spark]

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


##########
sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala:
##########
@@ -2570,7 +2570,7 @@ class ColumnExpressionSuite extends QueryTest with SharedSparkSession {
 
     assert(e3.getCause.isInstanceOf[RuntimeException])
     assert(e3.getCause.getMessage.matches(
-      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true!"))
+      "\\[USER_RAISED_EXCEPTION\\] '\\(a#\\d+ > b#\\d+\\)' is not true! SQLSTATE: P0001"))

Review Comment:
   Just wonder how such regexp should look like. Maybe put it to PR's description.



-- 
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-45595] Expose SQLSTATE in error message [spark]

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

   The stream failure is unrelated, merging to master. 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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/resources/error/error-classes.json:
##########
@@ -2039,6 +2039,12 @@
     },
     "sqlState" : "42K07"
   },
+  "INVALID_SCHEMA_OR_RELATION_NAME" : {
+    "message" : [
+      "`<name>` is not a valid name for tables/schemas. Valid names only contain alphabet characters, numbers and _."

Review Comment:
   Oops didn't see that when I flipped it.
   



-- 
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-45595] Expose SQLSTATE in error message [spark]

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


##########
common/utils/src/main/scala/org/apache/spark/SparkThrowableHelper.scala:
##########
@@ -51,9 +51,11 @@ private[spark] object SparkThrowableHelper {
       messageParameters: Map[String, String],
       context: String): String = {
     val displayMessage = errorReader.getErrorMessage(errorClass, messageParameters)
+    val sqlState = errorReader.getSqlState(errorClass)
+    val displaySqlState = if (sqlState == null) "" else s" SQLSTATE: $sqlState"
     val displayQueryContext = (if (context.isEmpty) "" else "\n") + context
     val prefix = if (errorClass.startsWith("_LEGACY_ERROR_")) "" else s"[$errorClass] "
-    s"$prefix$displayMessage$displayQueryContext"
+    s"$prefix$displayMessage$displaySqlState$displayQueryContext"

Review Comment:
   @srielau Could you a few sentences to PR's description about reasons for putting this at the end of error messages.



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