You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by ma...@apache.org on 2022/08/02 06:56:54 UTC

[spark] branch master updated: [SPARK-39935][SQL][TESTS] Switch `validateParsingError()` onto `checkError()`

This is an automated email from the ASF dual-hosted git repository.

maxgekk pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/master by this push:
     new 99fc389a0cd [SPARK-39935][SQL][TESTS] Switch `validateParsingError()` onto `checkError()`
99fc389a0cd is described below

commit 99fc389a0cd26056c4ad591c0aec70aac108ebe7
Author: Max Gekk <ma...@gmail.com>
AuthorDate: Tue Aug 2 11:56:33 2022 +0500

    [SPARK-39935][SQL][TESTS] Switch `validateParsingError()` onto `checkError()`
    
    ### What changes were proposed in this pull request?
    1. Re-implemented `validateParsingError()` using `checkError()`.
    2. Removed `checkParsingError()` and replaced by `checkError()`.
    
    ### Why are the changes needed?
    1. To prepare test infra for testing of query contexts.
    3. To check message parameters instead of entire text message. This PR is some kind of follow up of https://github.com/apache/spark/pull/36693 and https://github.com/apache/spark/pull/37322.
    
    ### Does this PR introduce _any_ user-facing change?
    No.
    
    ### How was this patch tested?
    By running the modified test suites:
    ```
    $ build/sbt -Phive -Phive-thriftserver "test:testOnly *TruncateTableParserSuite"
    $ build/sbt -Phive-2.3 -Phive-thriftserver "test:testOnly *ShowPartitionsParserSuite"
    $ build/sbt "sql/testOnly *QueryParsingErrorsSuite"
    ```
    
    Closes #37363 from MaxGekk/checkParsingError-to-checkError.
    
    Authored-by: Max Gekk <ma...@gmail.com>
    Signed-off-by: Max Gekk <ma...@gmail.com>
---
 .../spark/sql/errors/QueryErrorsSuiteBase.scala    |  31 +-
 .../spark/sql/errors/QueryParsingErrorsSuite.scala | 416 +++------------------
 .../command/ShowPartitionsParserSuite.scala        |  10 +-
 .../command/TruncateTableParserSuite.scala         |  10 +-
 4 files changed, 67 insertions(+), 400 deletions(-)

diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala
index d03c18882a4..525771f3038 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryErrorsSuiteBase.scala
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.errors
 
-import org.apache.spark.{QueryContext, SparkThrowable}
+import org.apache.spark.QueryContext
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.test.SharedSparkSession
 
@@ -28,28 +28,13 @@ trait QueryErrorsSuiteBase extends SharedSparkSession {
       errorClass: String,
       errorSubClass: Option[String] = None,
       sqlState: String,
-      message: String): Unit = {
-    val exception = intercept[ParseException] {
-      sql(sqlText)
-    }
-    checkParsingError(exception, errorClass, errorSubClass, sqlState, message)
-  }
-
-  def checkParsingError(
-      exception: Exception with SparkThrowable,
-      errorClass: String,
-      errorSubClass: Option[String] = None,
-      sqlState: String,
-      message: String): Unit = {
-    val fullErrorClass = if (errorSubClass.isDefined) {
-      errorClass + "." + errorSubClass.get
-    } else {
-      errorClass
-    }
-    assert(exception.getErrorClass === errorClass)
-    assert(exception.getErrorSubClass === errorSubClass.orNull)
-    assert(exception.getSqlState === sqlState)
-    assert(exception.getMessage === s"""\n[$fullErrorClass] """ + message)
+      parameters: Map[String, String] = Map.empty): Unit = {
+    checkError(
+      exception = intercept[ParseException](sql(sqlText)),
+      errorClass = errorClass,
+      errorSubClass = errorSubClass,
+      sqlState = Some(sqlState),
+      parameters = parameters)
   }
 
   case class ExpectedContext(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala
index 6dc40c3eadb..e9379b461ec 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryParsingErrorsSuite.scala
@@ -28,14 +28,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)",
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("LATERAL_NATURAL_JOIN"),
-      sqlState = "0A000",
-      message =
-        """The feature is not supported: NATURAL join with LATERAL correlation.(line 1, pos 14)
-          |
-          |== SQL ==
-          |SELECT * FROM t1 NATURAL JOIN LATERAL (SELECT c1 + c2 AS c2)
-          |--------------^^^
-          |""".stripMargin)
+      sqlState = "0A000")
   }
 
   test("UNSUPPORTED_FEATURE: LATERAL join with USING join not supported") {
@@ -43,14 +36,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)",
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("LATERAL_JOIN_USING"),
-      sqlState = "0A000",
-      message =
-        """The feature is not supported: JOIN USING with LATERAL correlation.(line 1, pos 14)
-          |
-          |== SQL ==
-          |SELECT * FROM t1 JOIN LATERAL (SELECT c1 + c2 AS c2) USING (c2)
-          |--------------^^^
-          |""".stripMargin)
+      sqlState = "0A000")
   }
 
   test("UNSUPPORTED_FEATURE: Unsupported LATERAL join type") {
@@ -60,13 +46,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
         errorClass = "UNSUPPORTED_FEATURE",
         errorSubClass = Some("LATERAL_JOIN_OF_TYPE"),
         sqlState = "0A000",
-        message =
-          s"""The feature is not supported: $joinType JOIN with LATERAL correlation.(line 1, pos 14)
-            |
-            |== SQL ==
-            |SELECT * FROM t1 $joinType JOIN LATERAL (SELECT c1 + c2 AS c3) ON c2 = c3
-            |--------------^^^
-            |""".stripMargin)
+        parameters = Map("joinType" -> joinType))
     }
   }
 
@@ -83,13 +63,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
         sqlText = sqlText,
         errorClass = "INVALID_SQL_SYNTAX",
         sqlState = "42000",
-        message =
-          s"""Invalid SQL syntax: LATERAL can only be used with subquery.(line 1, pos $pos)
-            |
-            |== SQL ==
-            |$sqlText
-            |${"-" * pos}^^^
-            |""".stripMargin)
+        parameters = Map("inputString" -> "LATERAL can only be used with subquery."))
     }
   }
 
@@ -98,14 +72,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT * FROM a NATURAL CROSS JOIN b",
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("NATURAL_CROSS_JOIN"),
-      sqlState = "0A000",
-      message =
-        """The feature is not supported: NATURAL CROSS JOIN.(line 1, pos 14)
-          |
-          |== SQL ==
-          |SELECT * FROM a NATURAL CROSS JOIN b
-          |--------------^^^
-          |""".stripMargin)
+      sqlState = "0A000")
   }
 
   test("INVALID_SQL_SYNTAX: redefine window") {
@@ -113,13 +80,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT min(a) OVER win FROM t1 WINDOW win AS win, win AS win2",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: The definition of window `win` is repetitive.(line 1, pos 31)
-          |
-          |== SQL ==
-          |SELECT min(a) OVER win FROM t1 WINDOW win AS win, win AS win2
-          |-------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "The definition of window `win` is repetitive."))
   }
 
   test("INVALID_SQL_SYNTAX: invalid window reference") {
@@ -127,13 +88,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT min(a) OVER win FROM t1 WINDOW win AS win",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Window reference `win` is not a window specification.(line 1, pos 31)
-          |
-          |== SQL ==
-          |SELECT min(a) OVER win FROM t1 WINDOW win AS win
-          |-------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Window reference `win` is not a window specification."))
   }
 
   test("INVALID_SQL_SYNTAX: window reference cannot be resolved") {
@@ -141,13 +96,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT min(a) OVER win FROM t1 WINDOW win AS win2",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Cannot resolve window reference `win2`.(line 1, pos 31)
-          |
-          |== SQL ==
-          |SELECT min(a) OVER win FROM t1 WINDOW win AS win2
-          |-------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Cannot resolve window reference `win2`."))
   }
 
   test("UNSUPPORTED_FEATURE: TRANSFORM does not support DISTINCT/ALL") {
@@ -155,14 +104,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT TRANSFORM(DISTINCT a) USING 'a' FROM t",
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("TRANSFORM_DISTINCT_ALL"),
-      sqlState = "0A000",
-      message =
-        """The feature is not supported: TRANSFORM with the DISTINCT/ALL clause.(line 1, pos 17)
-          |
-          |== SQL ==
-          |SELECT TRANSFORM(DISTINCT a) USING 'a' FROM t
-          |-----------------^^^
-          |""".stripMargin)
+      sqlState = "0A000")
   }
 
   test("UNSUPPORTED_FEATURE: In-memory mode does not support TRANSFORM with serde") {
@@ -171,14 +113,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
         "'org.apache.hadoop.hive.serde2.OpenCSVSerde' USING 'a' FROM t",
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("TRANSFORM_NON_HIVE"),
-      sqlState = "0A000",
-      message =
-        """The feature is not supported: TRANSFORM with SERDE is only supported in hive mode.(line 1, pos 0)
-          |
-          |== SQL ==
-          |SELECT TRANSFORM(a) ROW FORMAT SERDE 'org.apache.hadoop.hive.serde2.OpenCSVSerde' USING 'a' FROM t
-          |^^^
-          |""".stripMargin)
+      sqlState = "0A000")
   }
 
   test("INVALID_SQL_SYNTAX: Too many arguments for transform") {
@@ -186,13 +121,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "CREATE TABLE table(col int) PARTITIONED BY (years(col,col))",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Too many arguments for transform `years`(line 1, pos 44)
-          |
-          |== SQL ==
-          |CREATE TABLE table(col int) PARTITIONED BY (years(col,col))
-          |--------------------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Too many arguments for transform `years`"))
   }
 
   test("INVALID_SQL_SYNTAX: Invalid table value function name") {
@@ -200,26 +129,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SELECT * FROM db.func()",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: table valued function cannot specify database name (line 1, pos 14)
-          |
-          |== SQL ==
-          |SELECT * FROM db.func()
-          |--------------^^^
-          |""".stripMargin
-    )
+      parameters = Map("inputString" -> "table valued function cannot specify database name "))
 
     validateParsingError(
       sqlText = "SELECT * FROM ns.db.func()",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: table valued function cannot specify database name (line 1, pos 14)
-          |
-          |== SQL ==
-          |SELECT * FROM ns.db.func()
-          |--------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "table valued function cannot specify database name "))
   }
 
   test("INVALID_SQL_SYNTAX: Invalid scope in show functions") {
@@ -227,13 +143,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SHOW sys FUNCTIONS",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: SHOW `sys` FUNCTIONS not supported(line 1, pos 5)
-          |
-          |== SQL ==
-          |SHOW sys FUNCTIONS
-          |-----^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "SHOW `sys` FUNCTIONS not supported"))
   }
 
   test("INVALID_SQL_SYNTAX: Invalid pattern in show functions") {
@@ -241,24 +151,14 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SHOW FUNCTIONS IN db f1",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a "STRING" literal.(line 1, pos 21)
-          |
-          |== SQL ==
-          |SHOW FUNCTIONS IN db f1
-          |---------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" ->
+        "Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a \"STRING\" literal."))
     validateParsingError(
       sqlText = "SHOW FUNCTIONS IN db LIKE f1",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a "STRING" literal.(line 1, pos 26)
-          |
-          |== SQL ==
-          |SHOW FUNCTIONS IN db LIKE f1
-          |--------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" ->
+        "Invalid pattern in SHOW FUNCTIONS: `f1`. It must be a \"STRING\" literal."))
   }
 
   test("INVALID_SQL_SYNTAX: Create function with both if not exists and replace") {
@@ -268,23 +168,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
         |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
         |JAR '/path/to/jar2'
         |""".stripMargin
-    val errorDesc =
-      """CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed.(line 2, pos 0)"""
 
     validateParsingError(
       sqlText = sqlText,
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        s"""Invalid SQL syntax: $errorDesc
-          |
-          |== SQL ==
-          |
-          |CREATE OR REPLACE FUNCTION IF NOT EXISTS func1 as
-          |^^^
-          |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
-          |JAR '/path/to/jar2'
-          |""".stripMargin)
+      parameters = Map("inputString" ->
+        "CREATE FUNCTION with both IF NOT EXISTS and REPLACE is not allowed."))
   }
 
   test("INVALID_SQL_SYNTAX: Create temporary function with if not exists") {
@@ -294,23 +184,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
         |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
         |JAR '/path/to/jar2'
         |""".stripMargin
-    val errorDesc =
-      """It is not allowed to define a TEMPORARY FUNCTION with IF NOT EXISTS.(line 2, pos 0)"""
 
     validateParsingError(
       sqlText = sqlText,
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        s"""Invalid SQL syntax: $errorDesc
-          |
-          |== SQL ==
-          |
-          |CREATE TEMPORARY FUNCTION IF NOT EXISTS func1 as
-          |^^^
-          |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
-          |JAR '/path/to/jar2'
-          |""".stripMargin)
+      parameters = Map("inputString" ->
+        "It is not allowed to define a TEMPORARY FUNCTION with IF NOT EXISTS."))
   }
 
   test("INVALID_SQL_SYNTAX: Create temporary function with multi-part name") {
@@ -325,16 +205,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = sqlText,
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Unsupported function name `ns`.`db`.`func`(line 2, pos 0)
-          |
-          |== SQL ==
-          |
-          |CREATE TEMPORARY FUNCTION ns.db.func as
-          |^^^
-          |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
-          |JAR '/path/to/jar2'
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Unsupported function name `ns`.`db`.`func`"))
   }
 
   test("INVALID_SQL_SYNTAX: Specifying database while creating temporary function") {
@@ -344,40 +215,22 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
         |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
         |JAR '/path/to/jar2'
         |""".stripMargin
-    val errorDesc =
-      """Specifying a database in CREATE TEMPORARY FUNCTION is not allowed: `db`(line 2, pos 0)"""
 
     validateParsingError(
       sqlText = sqlText,
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        s"""Invalid SQL syntax: $errorDesc
-          |
-          |== SQL ==
-          |
-          |CREATE TEMPORARY FUNCTION db.func as
-          |^^^
-          |'com.matthewrathbone.example.SimpleUDFExample' USING JAR '/path/to/jar1',
-          |JAR '/path/to/jar2'
-          |""".stripMargin)
+      parameters = Map("inputString" ->
+        "Specifying a database in CREATE TEMPORARY FUNCTION is not allowed: `db`"))
   }
 
   test("INVALID_SQL_SYNTAX: Drop temporary function requires a single part name") {
-    val errorDesc =
-      "DROP TEMPORARY FUNCTION requires a single part name but got: `db`.`func`(line 1, pos 0)"
-
     validateParsingError(
       sqlText = "DROP TEMPORARY FUNCTION db.func",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        s"""Invalid SQL syntax: $errorDesc
-          |
-          |== SQL ==
-          |DROP TEMPORARY FUNCTION db.func
-          |^^^
-          |""".stripMargin)
+      parameters = Map("inputString" ->
+        "DROP TEMPORARY FUNCTION requires a single part name but got: `db`.`func`"))
   }
 
   test("DUPLICATE_KEY: Found duplicate partition keys") {
@@ -385,13 +238,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "INSERT OVERWRITE TABLE table PARTITION(p1='1', p1='1') SELECT 'col1', 'col2'",
       errorClass = "DUPLICATE_KEY",
       sqlState = "23000",
-      message =
-        """Found duplicate keys `p1`(line 1, pos 29)
-          |
-          |== SQL ==
-          |INSERT OVERWRITE TABLE table PARTITION(p1='1', p1='1') SELECT 'col1', 'col2'
-          |-----------------------------^^^
-          |""".stripMargin)
+      parameters = Map("keyColumn" -> "`p1`"))
   }
 
   test("DUPLICATE_KEY: in table properties") {
@@ -399,51 +246,24 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2')",
       errorClass = "DUPLICATE_KEY",
       sqlState = "23000",
-      message =
-        """Found duplicate keys `key1`(line 1, pos 39)
-          |
-          |== SQL ==
-          |ALTER TABLE dbx.tab1 SET TBLPROPERTIES ('key1' = '1', 'key1' = '2')
-          |---------------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("keyColumn" -> "`key1`"))
   }
 
   test("PARSE_EMPTY_STATEMENT: empty input") {
     validateParsingError(
       sqlText = "",
       errorClass = "PARSE_EMPTY_STATEMENT",
-      sqlState = "42000",
-      message =
-        """Syntax error, unexpected empty statement(line 1, pos 0)
-          |
-          |== SQL ==
-          |
-          |^^^
-          |""".stripMargin)
+      sqlState = "42000")
 
     validateParsingError(
       sqlText = "   ",
       errorClass = "PARSE_EMPTY_STATEMENT",
-      sqlState = "42000",
-      message =
-        s"""Syntax error, unexpected empty statement(line 1, pos 3)
-           |
-           |== SQL ==
-           |${"   "}
-           |---^^^
-           |""".stripMargin)
+      sqlState = "42000")
 
     validateParsingError(
       sqlText = " \n",
       errorClass = "PARSE_EMPTY_STATEMENT",
-      sqlState = "42000",
-      message =
-        s"""Syntax error, unexpected empty statement(line 2, pos 0)
-           |
-           |== SQL ==
-           |${" "}
-           |^^^
-           |""".stripMargin)
+      sqlState = "42000")
   }
 
   test("PARSE_SYNTAX_ERROR: no viable input") {
@@ -452,13 +272,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = sqlText,
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        s"""Syntax error at or near end of input(line 1, pos 16)
-          |
-          |== SQL ==
-          |$sqlText
-          |----------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "end of input", "hint" -> ""))
   }
 
   test("PARSE_SYNTAX_ERROR: extraneous input") {
@@ -466,26 +280,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "select 1 1",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near '1': extra input '1'(line 1, pos 9)
-          |
-          |== SQL ==
-          |select 1 1
-          |---------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "'1'", "hint" -> ": extra input '1'"))
 
     validateParsingError(
       sqlText = "select *\nfrom r as q t",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near 't': extra input 't'(line 2, pos 12)
-          |
-          |== SQL ==
-          |select *
-          |from r as q t
-          |------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "'t'", "hint" -> ": extra input 't'"))
   }
 
   test("PARSE_SYNTAX_ERROR: mismatched input") {
@@ -493,28 +294,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "select * from r order by q from t",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near 'from'(line 1, pos 27)
-          |
-          |== SQL ==
-          |select * from r order by q from t
-          |---------------------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "'from'", "hint" -> ""))
 
     validateParsingError(
       sqlText = "select *\nfrom r\norder by q\nfrom t",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near 'from'(line 4, pos 0)
-          |
-          |== SQL ==
-          |select *
-          |from r
-          |order by q
-          |from t
-          |^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "'from'", "hint" -> ""))
   }
 
   test("PARSE_SYNTAX_ERROR: jargon token substitute to user-facing language") {
@@ -523,25 +309,13 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "select count(*",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near end of input(line 1, pos 14)
-          |
-          |== SQL ==
-          |select count(*
-          |--------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "end of input", "hint" -> ""))
 
     validateParsingError(
       sqlText = "select 1 as a from",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near end of input(line 1, pos 18)
-          |
-          |== SQL ==
-          |select 1 as a from
-          |------------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "end of input", "hint" -> ""))
   }
 
   test("PARSE_SYNTAX_ERROR - SPARK-21136: " +
@@ -550,37 +324,19 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "select * from a left join_ b on a.id = b.id",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near 'join_': missing 'JOIN'(line 1, pos 21)
-          |
-          |== SQL ==
-          |select * from a left join_ b on a.id = b.id
-          |---------------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "'join_'", "hint" -> ": missing 'JOIN'"))
 
     validateParsingError(
       sqlText = "select * from test where test.t is like 'test'",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near 'is'(line 1, pos 32)
-          |
-          |== SQL ==
-          |select * from test where test.t is like 'test'
-          |--------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "'is'", "hint" -> ""))
 
     validateParsingError(
       sqlText = "SELECT * FROM test WHERE x NOT NULL",
       errorClass = "PARSE_SYNTAX_ERROR",
       sqlState = "42000",
-      message =
-        """Syntax error at or near 'NOT'(line 1, pos 27)
-          |
-          |== SQL ==
-          |SELECT * FROM test WHERE x NOT NULL
-          |---------------------------^^^
-          |""".stripMargin)
+      parameters = Map("error" -> "'NOT'", "hint" -> ""))
   }
 
   test("INVALID_SQL_SYNTAX: show table partition key must set value") {
@@ -588,13 +344,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "SHOW TABLE EXTENDED IN default LIKE 'employee' PARTITION (grade)",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Partition key `grade` must set value (can't be empty).(line 1, pos 47)
-          |
-          |== SQL ==
-          |SHOW TABLE EXTENDED IN default LIKE 'employee' PARTITION (grade)
-          |-----------------------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Partition key `grade` must set value (can't be empty)."))
   }
 
   test("INVALID_SQL_SYNTAX: expected a column reference for transform bucket") {
@@ -603,13 +353,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
         "CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (bucket(32, a, 66))",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Expected a column reference for transform `bucket`: 66(line 1, pos 67)
-          |
-          |== SQL ==
-          |CREATE TABLE my_tab(a INT, b STRING) USING parquet PARTITIONED BY (bucket(32, a, 66))
-          |-------------------------------------------------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Expected a column reference for transform `bucket`: 66"))
   }
 
   test("UNSUPPORTED_FEATURE: DESC TABLE COLUMN for a specific partition") {
@@ -617,16 +361,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "DESCRIBE TABLE EXTENDED customer PARTITION (grade = 'A') customer.age",
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("DESC_TABLE_COLUMN_PARTITION"),
-      sqlState = "0A000",
-      message =
-        """The feature is not supported: DESC TABLE COLUMN for a specific partition""" +
-        """.(line 1, pos 0)""" +
-        """|
-           |
-           |== SQL ==
-           |DESCRIBE TABLE EXTENDED customer PARTITION (grade = 'A') customer.age
-           |^^^
-           |""".stripMargin)
+      sqlState = "0A000")
   }
 
   test("INVALID_SQL_SYNTAX: PARTITION specification is incomplete") {
@@ -634,13 +369,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = "DESCRIBE TABLE EXTENDED customer PARTITION (grade)",
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: PARTITION specification is incomplete: `grade`(line 1, pos 0)
-          |
-          |== SQL ==
-          |DESCRIBE TABLE EXTENDED customer PARTITION (grade)
-          |^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "PARTITION specification is incomplete: `grade`"))
   }
 
   test("UNSUPPORTED_FEATURE: cannot set reserved namespace property") {
@@ -650,15 +379,9 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("SET_NAMESPACE_PROPERTY"),
       sqlState = "0A000",
-      message =
-        """The feature is not supported: location is a reserved namespace property, """ +
-        """please use the LOCATION clause to specify it.(line 1, pos 0)""" +
-        s"""
-          |
-          |== SQL ==
-          |$sql
-          |^^^
-          |""".stripMargin)
+      parameters = Map(
+        "property" -> "location",
+        "msg" -> "please use the LOCATION clause to specify it"))
   }
 
   test("UNSUPPORTED_FEATURE: cannot set reserved table property") {
@@ -669,15 +392,9 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("SET_TABLE_PROPERTY"),
       sqlState = "0A000",
-      message =
-        """The feature is not supported: provider is a reserved table property, """ +
-        """please use the USING clause to specify it.(line 1, pos 66)""" +
-        s"""
-          |
-          |== SQL ==
-          |$sql
-          |------------------------------------------------------------------^^^
-          |""".stripMargin)
+      parameters = Map(
+        "property" -> "provider",
+        "msg" -> "please use the USING clause to specify it"))
   }
 
   test("INVALID_PROPERTY_KEY: invalid property key for set quoted configuration") {
@@ -686,13 +403,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = sql,
       errorClass = "INVALID_PROPERTY_KEY",
       sqlState = null,
-      message =
-        s""""" is an invalid property key, please use quotes, e.g. SET ""="value"(line 1, pos 0)
-          |
-          |== SQL ==
-          |$sql
-          |^^^
-          |""".stripMargin)
+      parameters = Map("key" -> "\"\"", "value" -> "\"value\""))
   }
 
   test("INVALID_PROPERTY_VALUE: invalid property value for set quoted configuration") {
@@ -701,15 +412,7 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = sql,
       errorClass = "INVALID_PROPERTY_VALUE",
       sqlState = null,
-      message =
-        """"1;2;;" is an invalid property value, please use quotes, """ +
-        """e.g. SET "key"="1;2;;"(line 1, pos 0)""" +
-        s"""
-           |
-           |== SQL ==
-           |$sql
-           |^^^
-           |""".stripMargin)
+      parameters = Map("value" -> "\"1;2;;\"", "key" -> "\"key\""))
   }
 
   test("UNSUPPORTED_FEATURE: cannot set Properties and DbProperties at the same time") {
@@ -719,15 +422,6 @@ class QueryParsingErrorsSuite extends QueryTest with QueryErrorsSuiteBase {
       sqlText = sql,
       errorClass = "UNSUPPORTED_FEATURE",
       errorSubClass = Some("SET_PROPERTIES_AND_DBPROPERTIES"),
-      sqlState = "0A000",
-      message =
-        """The feature is not supported: set PROPERTIES and DBPROPERTIES at the same time.""" +
-        """(line 1, pos 0)""" +
-        s"""
-          |
-          |== SQL ==
-          |$sql
-          |^^^
-          |""".stripMargin)
+      sqlState = "0A000")
   }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala
index 8f75f5df7b9..c9cc5911484 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/ShowPartitionsParserSuite.scala
@@ -48,18 +48,12 @@ class ShowPartitionsParserSuite extends AnalysisTest with QueryErrorsSuiteBase {
   }
 
   test("empty values in non-optional partition specs") {
-    checkParsingError(
+    checkError(
       exception = intercept[ParseException] {
         new SparkSqlParser().parsePlan("SHOW PARTITIONS dbx.tab1 PARTITION (a='1', b)")
       },
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Partition key `b` must set value (can't be empty).(line 1, pos 25)
-          |
-          |== SQL ==
-          |SHOW PARTITIONS dbx.tab1 PARTITION (a='1', b)
-          |-------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Partition key `b` must set value (can't be empty)."))
   }
 }
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala
index 761e4222792..ef45cdebbe8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/TruncateTableParserSuite.scala
@@ -47,18 +47,12 @@ class TruncateTableParserSuite extends AnalysisTest with QueryErrorsSuiteBase {
   }
 
   test("empty values in non-optional partition specs") {
-    checkParsingError(
+    checkError(
       exception = intercept[ParseException] {
         parsePlan("TRUNCATE TABLE dbx.tab1 PARTITION (a='1', b)")
       },
       errorClass = "INVALID_SQL_SYNTAX",
       sqlState = "42000",
-      message =
-        """Invalid SQL syntax: Partition key `b` must set value (can't be empty).(line 1, pos 24)
-          |
-          |== SQL ==
-          |TRUNCATE TABLE dbx.tab1 PARTITION (a='1', b)
-          |------------------------^^^
-          |""".stripMargin)
+      parameters = Map("inputString" -> "Partition key `b` must set value (can't be empty)."))
   }
 }


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org
For additional commands, e-mail: commits-help@spark.apache.org