You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by GitBox <gi...@apache.org> on 2022/11/22 08:49:06 UTC

[GitHub] [spark] LuciferYang opened a new pull request, #38754: [SPAKR-41180][SQL] Assign an error class to "Cannot parse the data type"

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

   ### What changes were proposed in this pull request?
   This pr aims rename `_LEGACY_ERROR_TEMP_1227` to `INVALID_DATA_TYPE_SCHEMA`
   
   
   ### Why are the changes needed?
   Proper names of error classes to improve user experience with Spark SQL.
   
   
   ### Does this PR introduce _any_ user-facing change?
   No
   
   
   ### How was this patch tested?
   Pass GitHub Actions


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


[GitHub] [spark] LuciferYang commented on a diff in pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033358837


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala:
##########
@@ -160,8 +157,8 @@ object DataType {
         try {
           fallbackParser(schema)
         } catch {
-          case NonFatal(e2) =>
-            throw QueryCompilationErrors.failedFallbackParsingError(errorMsg, e1, e2)
+          case NonFatal(_) =>

Review Comment:
   Do you mean if `e2` is `NonFatal ` and `e1` is  `SparkThrowable`, then re-throw it,  and only call `QueryCompilationErrors.schemaFailToParseError` when e1 is not `SparkThrowable`?
   



##########
sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala:
##########
@@ -160,8 +157,8 @@ object DataType {
         try {
           fallbackParser(schema)
         } catch {
-          case NonFatal(e2) =>
-            throw QueryCompilationErrors.failedFallbackParsingError(errorMsg, e1, e2)
+          case NonFatal(_) =>

Review Comment:
   Do you mean if `e2` is `NonFatal ` and `e1` is  `SparkThrowable`, then re-throw it,  and only call `QueryCompilationErrors.schemaFailToParseError` when `e1` is not `SparkThrowable`?
   



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


[GitHub] [spark] LuciferYang commented on a diff in pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033411176


##########
sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala:
##########
@@ -443,19 +443,31 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
       exception = intercept[AnalysisException] {
         df3.selectExpr("from_json(value, 1)")
       },
-      errorClass = "INVALID_SCHEMA",
-      parameters = Map("expr" -> "\"1\""),
+      errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL",
+      parameters = Map("inputSchema" -> "\"1\""),
       context = ExpectedContext(
         fragment = "from_json(value, 1)",
         start = 0,
         stop = 18
       )
     )
 
-    val errMsg2 = intercept[AnalysisException] {
-      df3.selectExpr("""from_json(value, 'time InvalidType')""")
-    }
-    assert(errMsg2.getMessage.contains("DataType invalidtype is not supported"))
+    checkError(

Review Comment:
   The case corresponding to `e` is `SparkThrowable`
   
   



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


[GitHub] [spark] MaxGekk closed pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
MaxGekk closed pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`
URL: https://github.com/apache/spark/pull/38754


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


[GitHub] [spark] LuciferYang commented on a diff in pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033411900


##########
sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala:
##########
@@ -991,22 +1003,47 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
   test("SPARK-33286: from_json - combined error messages") {
     val df = Seq("""{"a":1}""").toDF("json")
     val invalidJsonSchema = """{"fields": [{"a":123}], "type": "struct"}"""
-    val errMsg1 = intercept[AnalysisException] {
-      df.select(from_json($"json", invalidJsonSchema, Map.empty[String, String])).collect()
-    }.getMessage
-    assert(errMsg1.contains("""Failed to convert the JSON string '{"a":123}' to a field"""))
+    val invalidJsonSchemaReason = "Failed to convert the JSON string '{\"a\":123}' to a field."

Review Comment:
   The following two case corresponding to `e` is  not SparkThrowable



##########
sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala:
##########
@@ -991,22 +1003,47 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
   test("SPARK-33286: from_json - combined error messages") {
     val df = Seq("""{"a":1}""").toDF("json")
     val invalidJsonSchema = """{"fields": [{"a":123}], "type": "struct"}"""
-    val errMsg1 = intercept[AnalysisException] {
-      df.select(from_json($"json", invalidJsonSchema, Map.empty[String, String])).collect()
-    }.getMessage
-    assert(errMsg1.contains("""Failed to convert the JSON string '{"a":123}' to a field"""))
+    val invalidJsonSchemaReason = "Failed to convert the JSON string '{\"a\":123}' to a field."

Review Comment:
   The following two case corresponding to `e` is  not `SparkThrowable`



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


[GitHub] [spark] LuciferYang commented on a diff in pull request #38754: [WIP][SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033070950


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -780,8 +780,21 @@
   },
   "INVALID_SCHEMA" : {
     "message" : [
-      "The expression <expr> is not a valid schema string."
-    ]
+      "The input schema <inputSchema> is not a valid schema string."
+    ],
+    "subClass" : {
+      "PARSE_ERROR" : {
+        "message" : [
+          "Cannot parse the schema: ",

Review Comment:
   Thanks



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -780,8 +780,21 @@
   },
   "INVALID_SCHEMA" : {
     "message" : [
-      "The expression <expr> is not a valid schema string."
-    ]
+      "The input schema <inputSchema> is not a valid schema string."
+    ],
+    "subClass" : {
+      "PARSE_ERROR" : {
+        "message" : [
+          "Cannot parse the schema: ",
+          "<reason>"
+        ]
+      },
+      "UNEXPECTED_INPUT_TYPE" : {

Review Comment:
   renamed



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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38754: [WIP][SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033000254


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -780,8 +780,21 @@
   },
   "INVALID_SCHEMA" : {
     "message" : [
-      "The expression <expr> is not a valid schema string."
-    ]
+      "The input schema <inputSchema> is not a valid schema string."
+    ],
+    "subClass" : {
+      "PARSE_ERROR" : {
+        "message" : [
+          "Cannot parse the schema: ",
+          "<reason>"
+        ]
+      },
+      "UNEXPECTED_INPUT_TYPE" : {

Review Comment:
   How about `NON_STRING_LITERAL`?



##########
core/src/main/resources/error/error-classes.json:
##########
@@ -780,8 +780,21 @@
   },
   "INVALID_SCHEMA" : {
     "message" : [
-      "The expression <expr> is not a valid schema string."
-    ]
+      "The input schema <inputSchema> is not a valid schema string."
+    ],
+    "subClass" : {
+      "PARSE_ERROR" : {
+        "message" : [
+          "Cannot parse the schema: ",

Review Comment:
   ```suggestion
             "Cannot parse the schema:",
   ```



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


[GitHub] [spark] MaxGekk commented on pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on PR #38754:
URL: https://github.com/apache/spark/pull/38754#issuecomment-1329471491

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


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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38754: [SPARK-41180][SQL] Assign an error class to "Cannot parse the data type"

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1032769011


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -641,6 +641,12 @@
     ],
     "sqlState" : "42000"
   },
+  "INVALID_DATA_TYPE_SCHEMA" : {

Review Comment:
   We already have the error class `INVALID_SCHEMA`. Can you re-use it or make it more generic? cc @srielau 



##########
sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out:
##########
@@ -43,7 +43,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_1227",
+  "errorClass" : "INVALID_DATA_TYPE_SCHEMA",
   "messageParameters" : {
     "e1" : "\n[PARSE_SYNTAX_ERROR] Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2)\n\n== SQL ==\na InvalidType\n--^^^\n",

Review Comment:
   How about to unwrap this error? When both attempts fail, just bypass the first one to users.



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


[GitHub] [spark] LuciferYang commented on a diff in pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033411900


##########
sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala:
##########
@@ -991,22 +1003,47 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
   test("SPARK-33286: from_json - combined error messages") {
     val df = Seq("""{"a":1}""").toDF("json")
     val invalidJsonSchema = """{"fields": [{"a":123}], "type": "struct"}"""
-    val errMsg1 = intercept[AnalysisException] {
-      df.select(from_json($"json", invalidJsonSchema, Map.empty[String, String])).collect()
-    }.getMessage
-    assert(errMsg1.contains("""Failed to convert the JSON string '{"a":123}' to a field"""))
+    val invalidJsonSchemaReason = "Failed to convert the JSON string '{\"a\":123}' to a field."

Review Comment:
   The following three case corresponding to `e` is  not `SparkThrowable`



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


[GitHub] [spark] LuciferYang commented on pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on PR #38754:
URL: https://github.com/apache/spark/pull/38754#issuecomment-1328094947

   two test case failed, will fix later


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


[GitHub] [spark] LuciferYang commented on a diff in pull request #38754: [WIP][SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033071448


##########
core/src/main/resources/error/error-classes.json:
##########
@@ -780,8 +780,21 @@
   },
   "INVALID_SCHEMA" : {
     "message" : [
-      "The expression <expr> is not a valid schema string."
-    ]
+      "The input schema <inputSchema> is not a valid schema string."
+    ],
+    "subClass" : {
+      "PARSE_ERROR" : {
+        "message" : [
+          "Cannot parse the schema: ",
+          "<reason>"
+        ]
+      },
+      "UNEXPECTED_INPUT_TYPE" : {

Review Comment:
   [af157d2](https://github.com/apache/spark/pull/38754/commits/af157d2bf34489ebbd116f6077eb13c5287971cc) rename ` INVALID_SCHEMA.UNEXPECTED_INPUT_TYPE` to `INVALID_SCHEMA.NON_STRING_LITERAL`



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


[GitHub] [spark] LuciferYang commented on pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on PR #38754:
URL: https://github.com/apache/spark/pull/38754#issuecomment-1329958569

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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033344766


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala:
##########
@@ -160,8 +157,8 @@ object DataType {
         try {
           fallbackParser(schema)
         } catch {
-          case NonFatal(e2) =>
-            throw QueryCompilationErrors.failedFallbackParsingError(errorMsg, e1, e2)
+          case NonFatal(_) =>

Review Comment:
   Could you catch `SparkThrowable`, and re-throw it.



##########
sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out:
##########
@@ -43,11 +43,10 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 {
-  "errorClass" : "_LEGACY_ERROR_TEMP_1227",
+  "errorClass" : "INVALID_SCHEMA.PARSE_ERROR",
   "messageParameters" : {
-    "e1" : "\n[PARSE_SYNTAX_ERROR] Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2)\n\n== SQL ==\na InvalidType\n--^^^\n",
-    "e2" : "\nDataType invalidtype is not supported.(line 1, pos 2)\n\n== SQL ==\na InvalidType\n--^^^\n",
-    "msg" : "Cannot parse the data type: "
+    "inputSchema" : "\"a InvalidType\"",
+    "reason" : "\n[PARSE_SYNTAX_ERROR] Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2)\n\n== SQL ==\na InvalidType\n--^^^\n"

Review Comment:
   Let's bypass `SparkThrowable` to users. I don't think the wrapping is useful.



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


[GitHub] [spark] MaxGekk commented on a diff in pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033374978


##########
sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala:
##########
@@ -160,8 +157,8 @@ object DataType {
         try {
           fallbackParser(schema)
         } catch {
-          case NonFatal(e2) =>
-            throw QueryCompilationErrors.failedFallbackParsingError(errorMsg, e1, e2)
+          case NonFatal(_) =>

Review Comment:
   Yep



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


[GitHub] [spark] LuciferYang commented on a diff in pull request #38754: [SPARK-41180][SQL] Reuse `INVALID_SCHEMA` instead of `_LEGACY_ERROR_TEMP_1227`

Posted by GitBox <gi...@apache.org>.
LuciferYang commented on code in PR #38754:
URL: https://github.com/apache/spark/pull/38754#discussion_r1033411176


##########
sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala:
##########
@@ -443,19 +443,31 @@ class JsonFunctionsSuite extends QueryTest with SharedSparkSession {
       exception = intercept[AnalysisException] {
         df3.selectExpr("from_json(value, 1)")
       },
-      errorClass = "INVALID_SCHEMA",
-      parameters = Map("expr" -> "\"1\""),
+      errorClass = "INVALID_SCHEMA.NON_STRING_LITERAL",
+      parameters = Map("inputSchema" -> "\"1\""),
       context = ExpectedContext(
         fragment = "from_json(value, 1)",
         start = 0,
         stop = 18
       )
     )
 
-    val errMsg2 = intercept[AnalysisException] {
-      df3.selectExpr("""from_json(value, 'time InvalidType')""")
-    }
-    assert(errMsg2.getMessage.contains("DataType invalidtype is not supported"))
+    checkError(

Review Comment:
   This case corresponding to `e` is `SparkThrowable`
   
   



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


[GitHub] [spark] MaxGekk commented on pull request #38754: [SPARK-41180][SQL] Assign an error class to "Cannot parse the data type"

Posted by GitBox <gi...@apache.org>.
MaxGekk commented on PR #38754:
URL: https://github.com/apache/spark/pull/38754#issuecomment-1325376097

   @LuciferYang Could you resolve conflicts, please.


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