You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by we...@apache.org on 2022/03/22 15:22:02 UTC

[spark] branch branch-3.3 updated: [SPARK-38456][SQL] Improve error messages of no viable alternative, extraneous input and missing token in ParseException

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

wenchen pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/spark.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 0265c86  [SPARK-38456][SQL] Improve error messages of no viable alternative, extraneous input and missing token in ParseException
0265c86 is described below

commit 0265c86a0c735f7b39b1dac4a549bdf1e70d18c6
Author: Xinyi Yu <xi...@databricks.com>
AuthorDate: Tue Mar 22 23:18:34 2022 +0800

    [SPARK-38456][SQL] Improve error messages of no viable alternative, extraneous input and missing token in ParseException
    
    ### What changes were proposed in this pull request?
    This PR improves the "no viable alternative", "extraneous input" and "missing .. at " ANTLR error messages in ParseExceptions, as mentioned in https://issues.apache.org/jira/browse/SPARK-38456
    
    **With this PR, all ANTLR exceptions are unified to the same error class, `PARSE_SYNTAX_ERROR`.**
    
    #### No viable alternative
    * Query
        ```sql
        select (
        ```
    * Before
        ```
        no viable alternative at input ‘(‘(line 1, pos 8)
        ```
    * After
        ```
        Syntax error at or near end of input(line 1, pos 8)
        ```
    
    #### Extraneous Input
    * Query
        ```sql
        CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet
        ```
    * Before
        ```
        extraneous input ':' expecting {'APPLY', 'CALLED', 'CHANGES', 'CLONE', 'COLLECT', 'CONTAINS', 'CONVERT', 'COPY', 'COPY_OPTIONS', 'CREDENTIAL', 'CREDENTIALS', 'DEEP', 'DEFINER', 'DELTA', 'DETERMINISTIC', 'ENCRYPTION', 'EXPECT', 'FAIL', 'FILES', 'FORMAT_OPTIONS', 'HISTORY', 'INCREMENTAL', 'INPUT', 'INVOKER', 'LANGUAGE', 'LIVE', 'MATERIALIZED', 'MODIFIES', 'OPTIMIZE', 'PATTERN', 'READS', 'RESTORE', 'RETURN', 'RETURNS', 'SAMPLE', 'SCD TYPE 1', 'SCD TYPE 2', 'SECURITY', 'SEQUENCE', 'SH [...]
        ```
    * After
        ```
        Syntax error at or near ':': extra input ':'(line 1, pos 21)
        ```
    
    #### Missing token
    * Query
        ```sql
        select count(a from b
        ```
    * Before
        ```
        missing ')' at 'from'(line 2, pos 0)
        ```
    * After
        ```
        Syntax error at or near 'from': missing ')'(line 2, pos 0)
        ```
    
    ### Why are the changes needed?
    https://issues.apache.org/jira/browse/SPARK-38384 The description states the reason for the change.
    TLDR, the error messages of ParseException directly coming from ANTLR are not user-friendly and we want to improve it.
    
    ### Does this PR introduce _any_ user-facing change?
    If the error messages changes are considered as user-facing change, then yes.
    Example cases are listed in the top of this PR description.
    
    ### How was this patch tested?
    Local unit test.
    
    Closes #35915 from anchovYu/rest-parse-exceptions.
    
    Authored-by: Xinyi Yu <xi...@databricks.com>
    Signed-off-by: Wenchen Fan <we...@databricks.com>
    (cherry picked from commit 27455aee8e6c671dcfee757771be6cdd58c9b001)
    Signed-off-by: Wenchen Fan <we...@databricks.com>
---
 core/src/main/resources/error/error-classes.json   |  4 +-
 docs/sql-distributed-sql-engine-spark-sql-cli.md   |  2 +-
 docs/sql-ref-identifier.md                         |  4 +-
 .../spark/sql/catalyst/parser/ParseDriver.scala    |  4 +-
 .../catalyst/parser/SparkParserErrorStrategy.scala | 55 +++++++++++++++++-----
 .../spark/sql/catalyst/parser/DDLParserSuite.scala |  6 +--
 .../sql/catalyst/parser/ErrorParserSuite.scala     | 26 +++++-----
 .../catalyst/parser/ExpressionParserSuite.scala    |  8 ++--
 .../sql/catalyst/parser/PlanParserSuite.scala      | 22 ++++-----
 .../parser/TableIdentifierParserSuite.scala        |  2 +-
 .../sql-tests/results/ansi/interval.sql.out        |  4 +-
 .../sql-tests/results/ansi/literals.sql.out        |  2 +-
 .../sql-tests/results/csv-functions.sql.out        |  2 +-
 .../test/resources/sql-tests/results/cte.sql.out   |  2 +-
 .../sql-tests/results/grouping_set.sql.out         |  4 +-
 .../resources/sql-tests/results/interval.sql.out   |  4 +-
 .../sql-tests/results/json-functions.sql.out       |  2 +-
 .../resources/sql-tests/results/literals.sql.out   |  2 +-
 .../results/postgreSQL/window_part3.sql.out        |  2 +-
 .../resources/sql-tests/results/transform.sql.out  |  6 +--
 .../AlterTableRecoverPartitionsParserSuite.scala   |  2 +-
 .../execution/command/PlanResolutionSuite.scala    |  2 +-
 .../org/apache/spark/sql/jdbc/JDBCWriteSuite.scala |  2 +-
 .../spark/sql/hive/thriftserver/CliSuite.scala     |  2 +-
 24 files changed, 102 insertions(+), 69 deletions(-)

diff --git a/core/src/main/resources/error/error-classes.json b/core/src/main/resources/error/error-classes.json
index c7a9c85..cd47d50 100644
--- a/core/src/main/resources/error/error-classes.json
+++ b/core/src/main/resources/error/error-classes.json
@@ -142,8 +142,8 @@
     "message" : [ "Syntax error, unexpected empty statement" ],
     "sqlState" : "42000"
   },
-  "PARSE_INPUT_MISMATCHED" : {
-    "message" : [ "Syntax error at or near %s" ],
+  "PARSE_SYNTAX_ERROR" : {
+    "message" : [ "Syntax error at or near %s%s" ],
     "sqlState" : "42000"
   },
   "PIVOT_VALUE_DATA_TYPE_MISMATCH" : {
diff --git a/docs/sql-distributed-sql-engine-spark-sql-cli.md b/docs/sql-distributed-sql-engine-spark-sql-cli.md
index f7f3669..c417153 100644
--- a/docs/sql-distributed-sql-engine-spark-sql-cli.md
+++ b/docs/sql-distributed-sql-engine-spark-sql-cli.md
@@ -113,7 +113,7 @@ Use `;` (semicolon) to terminate commands. Notice:
    /* This is a comment contains ;
    */ SELECT 1;
    ```
-   However, if ';' is the end of the line, it terminates the SQL statement. The example above will be terminated into  `/* This is a comment contains ` and `*/ SELECT 1`, Spark will submit these two commands separated and throw parser error (`unclosed bracketed comment` and `extraneous input '*/'`).
+   However, if ';' is the end of the line, it terminates the SQL statement. The example above will be terminated into  `/* This is a comment contains ` and `*/ SELECT 1`, Spark will submit these two commands separated and throw parser error (`unclosed bracketed comment` and `Syntax error at or near '*/'`).
 
 <table class="table">
 <tr><th>Command</th><th>Description</th></tr>
diff --git a/docs/sql-ref-identifier.md b/docs/sql-ref-identifier.md
index f65d491..bba8c67 100644
--- a/docs/sql-ref-identifier.md
+++ b/docs/sql-ref-identifier.md
@@ -58,7 +58,7 @@ An identifier is a string used to identify a database object such as a table, vi
 -- This CREATE TABLE fails with ParseException because of the illegal identifier name a.b
 CREATE TABLE test (a.b int);
 org.apache.spark.sql.catalyst.parser.ParseException:
-no viable alternative at input 'CREATE TABLE test (a.'(line 1, pos 20)
+Syntax error at or near '.': extra input '.'(line 1, pos 20)
 
 -- This CREATE TABLE works
 CREATE TABLE test (`a.b` int);
@@ -66,7 +66,7 @@ CREATE TABLE test (`a.b` int);
 -- This CREATE TABLE fails with ParseException because special character ` is not escaped
 CREATE TABLE test1 (`a`b` int);
 org.apache.spark.sql.catalyst.parser.ParseException:
-no viable alternative at input 'CREATE TABLE test (`a`b`'(line 1, pos 23)
+Syntax error at or near '`'(line 1, pos 23)
 
 -- This CREATE TABLE works
 CREATE TABLE test (`a``b` int);
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
index 5c9c382..82be4d6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
@@ -289,8 +289,8 @@ class ParseException(
   }
 
   def withCommand(cmd: String): ParseException = {
-    // PARSE_EMPTY_STATEMENT error class overrides the PARSE_INPUT_MISMATCHED when cmd is empty
-    if (cmd.trim().isEmpty && errorClass.isDefined && errorClass.get == "PARSE_INPUT_MISMATCHED") {
+    // PARSE_EMPTY_STATEMENT error class overrides the PARSE_SYNTAX_ERROR when cmd is empty
+    if (cmd.trim().isEmpty && errorClass.isDefined && errorClass.get == "PARSE_SYNTAX_ERROR") {
       new ParseException(Option(cmd), start, stop, "PARSE_EMPTY_STATEMENT", Array[String]())
     } else {
       new ParseException(Option(cmd), message, start, stop, errorClass, messageParameters)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala
index 0ce514c..1b0b686 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala
@@ -17,8 +17,7 @@
 
 package org.apache.spark.sql.catalyst.parser
 
-import org.antlr.v4.runtime.{DefaultErrorStrategy, InputMismatchException, IntStream, Parser,
-  ParserRuleContext, RecognitionException, Recognizer}
+import org.antlr.v4.runtime.{DefaultErrorStrategy, InputMismatchException, IntStream, NoViableAltException, Parser, ParserRuleContext, RecognitionException, Recognizer, Token}
 
 /**
  * A [[SparkRecognitionException]] extends the [[RecognitionException]] with more information
@@ -49,6 +48,10 @@ class SparkRecognitionException(
       },
       Some(errorClass),
       messageParameters)
+
+  /** Construct with pure errorClass and messageParameter information.  */
+  def this(errorClass: String, messageParameters: Array[String]) =
+    this("", null, null, null, Some(errorClass), messageParameters)
 }
 
 /**
@@ -61,21 +64,49 @@ class SparkRecognitionException(
  */
 class SparkParserErrorStrategy() extends DefaultErrorStrategy {
   private val userWordDict : Map[String, String] = Map("'<EOF>'" -> "end of input")
-  private def getUserFacingLanguage(input: String) = {
-    userWordDict.getOrElse(input, input)
+
+  /** Get the user-facing display of the error token. */
+  override def getTokenErrorDisplay(t: Token): String = {
+    val tokenName = super.getTokenErrorDisplay(t)
+    userWordDict.getOrElse(tokenName, tokenName)
   }
 
   override def reportInputMismatch(recognizer: Parser, e: InputMismatchException): Unit = {
-    // Keep the original error message in ANTLR
-    val msg = "mismatched input " +
-      this.getTokenErrorDisplay(e.getOffendingToken) +
-      " expecting " +
-      e.getExpectedTokens.toString(recognizer.getVocabulary)
+    val exceptionWithErrorClass = new SparkRecognitionException(
+      e,
+      "PARSE_SYNTAX_ERROR",
+      Array(getTokenErrorDisplay(e.getOffendingToken), ""))
+    recognizer.notifyErrorListeners(e.getOffendingToken, "", exceptionWithErrorClass)
+  }
 
+  override def reportNoViableAlternative(recognizer: Parser, e: NoViableAltException): Unit = {
     val exceptionWithErrorClass = new SparkRecognitionException(
       e,
-      "PARSE_INPUT_MISMATCHED",
-      Array(getUserFacingLanguage(getTokenErrorDisplay(e.getOffendingToken))))
-    recognizer.notifyErrorListeners(e.getOffendingToken, msg, exceptionWithErrorClass)
+      "PARSE_SYNTAX_ERROR",
+      Array(getTokenErrorDisplay(e.getOffendingToken), ""))
+    recognizer.notifyErrorListeners(e.getOffendingToken, "", exceptionWithErrorClass)
+  }
+
+  override def reportUnwantedToken(recognizer: Parser): Unit = {
+    if (!this.inErrorRecoveryMode(recognizer)) {
+      this.beginErrorCondition(recognizer)
+      val errorTokenDisplay = getTokenErrorDisplay(recognizer.getCurrentToken)
+      val hint = ": extra input " + errorTokenDisplay
+      val exceptionWithErrorClass = new SparkRecognitionException(
+        "PARSE_SYNTAX_ERROR",
+        Array(errorTokenDisplay, hint))
+      recognizer.notifyErrorListeners(recognizer.getCurrentToken, "", exceptionWithErrorClass)
+    }
+  }
+
+  override def reportMissingToken(recognizer: Parser): Unit = {
+    if (!this.inErrorRecoveryMode(recognizer)) {
+      this.beginErrorCondition(recognizer)
+      val hint = ": missing " + getExpectedTokens(recognizer).toString(recognizer.getVocabulary)
+      val exceptionWithErrorClass = new SparkRecognitionException(
+        "PARSE_SYNTAX_ERROR",
+        Array(getTokenErrorDisplay(recognizer.getCurrentToken), hint))
+      recognizer.notifyErrorListeners(recognizer.getCurrentToken, "", exceptionWithErrorClass)
+    }
   }
 }
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
index 507b17b..d5d90ce 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
@@ -74,7 +74,7 @@ class DDLParserSuite extends AnalysisTest {
     }
 
     intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet",
-      "extraneous input ':'")
+      "Syntax error at or near ':': extra input ':'")
   }
 
   test("create/replace table - with IF NOT EXISTS") {
@@ -1777,9 +1777,9 @@ class DDLParserSuite extends AnalysisTest {
         allColumns = true))
 
     intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value",
-      Some("PARSE_INPUT_MISMATCHED"), "Syntax error at or near 'key'") // expecting {<EOF>, ';'}
+      Some("PARSE_SYNTAX_ERROR"), "Syntax error at or near 'key'") // expecting {<EOF>, ';'}
     intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL",
-      "missing 'COLUMNS' at '<EOF>'")
+      "Syntax error at or near end of input: missing 'COLUMNS'")
   }
 
   test("LOAD DATA INTO table") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
index 71296f0..20e17a8 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
@@ -79,21 +79,22 @@ class ErrorParserSuite extends AnalysisTest {
 
   test("no viable input") {
     intercept("select ((r + 1) ", 1, 16, 16,
-      "no viable alternative at input", "----------------^^^")
+      "Syntax error at or near", "----------------^^^")
   }
 
   test("extraneous input") {
-    intercept("select 1 1", 1, 9, 10, "extraneous input '1' expecting", "---------^^^")
-    intercept("select *\nfrom r as q t", 2, 12, 13, "extraneous input", "------------^^^")
+    intercept("select 1 1", 1, 9, 10,
+      "Syntax error at or near '1': extra input '1'", "---------^^^")
+    intercept("select *\nfrom r as q t", 2, 12, 13, "Syntax error at or near", "------------^^^")
   }
 
   test("mismatched input") {
-    intercept("select * from r order by q from t", "PARSE_INPUT_MISMATCHED",
+    intercept("select * from r order by q from t", "PARSE_SYNTAX_ERROR",
       1, 27, 31,
       "Syntax error at or near",
       "---------------------------^^^"
     )
-    intercept("select *\nfrom r\norder by q\nfrom t", "PARSE_INPUT_MISMATCHED",
+    intercept("select *\nfrom r\norder by q\nfrom t", "PARSE_SYNTAX_ERROR",
       4, 0, 4,
       "Syntax error at or near", "^^^")
   }
@@ -107,9 +108,9 @@ class ErrorParserSuite extends AnalysisTest {
 
   test("jargon token substitute to user-facing language") {
     // '<EOF>' -> end of input
-    intercept("select count(*", "PARSE_INPUT_MISMATCHED",
+    intercept("select count(*", "PARSE_SYNTAX_ERROR",
       1, 14, 14, "Syntax error at or near end of input")
-    intercept("select 1 as a from", "PARSE_INPUT_MISMATCHED",
+    intercept("select 1 as a from", "PARSE_SYNTAX_ERROR",
       1, 18, 18, "Syntax error at or near end of input")
   }
 
@@ -120,11 +121,12 @@ class ErrorParserSuite extends AnalysisTest {
   }
 
   test("SPARK-21136: misleading error message due to problematic antlr grammar") {
-    intercept("select * from a left join_ b on a.id = b.id", None, "missing 'JOIN' at 'join_'")
-    intercept("select * from test where test.t is like 'test'", Some("PARSE_INPUT_MISMATCHED"),
-      SparkThrowableHelper.getMessage("PARSE_INPUT_MISMATCHED", Array("'is'")))
-    intercept("SELECT * FROM test WHERE x NOT NULL", Some("PARSE_INPUT_MISMATCHED"),
-      SparkThrowableHelper.getMessage("PARSE_INPUT_MISMATCHED", Array("'NOT'")))
+    intercept("select * from a left join_ b on a.id = b.id", None,
+      "Syntax error at or near 'join_': missing 'JOIN'")
+    intercept("select * from test where test.t is like 'test'", Some("PARSE_SYNTAX_ERROR"),
+      SparkThrowableHelper.getMessage("PARSE_SYNTAX_ERROR", Array("'is'", "")))
+    intercept("SELECT * FROM test WHERE x NOT NULL", Some("PARSE_SYNTAX_ERROR"),
+      SparkThrowableHelper.getMessage("PARSE_SYNTAX_ERROR", Array("'NOT'", "")))
   }
 
   test("hyphen in identifier - DDL tests") {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
index 754ac8b..9e63c81 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala
@@ -284,7 +284,7 @@ class ExpressionParserSuite extends AnalysisTest {
     assertEqual("foo(distinct a, b)", 'foo.distinctFunction('a, 'b))
     assertEqual("grouping(distinct a, b)", 'grouping.distinctFunction('a, 'b))
     assertEqual("`select`(all a, b)", 'select.function('a, 'b))
-    intercept("foo(a x)", "extraneous input 'x'")
+    intercept("foo(a x)", "Syntax error at or near 'x': extra input 'x'")
   }
 
   private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name))
@@ -654,7 +654,7 @@ class ExpressionParserSuite extends AnalysisTest {
           // Note: Single quote follows 1.6 parsing behavior
           // when ESCAPED_STRING_LITERALS is enabled.
           val e = intercept[ParseException](parser.parseExpression("'\''"))
-          assert(e.message.contains("extraneous input '''"))
+          assert(e.message.contains("Syntax error at or near ''': extra input '''"))
 
           // The unescape special characters (e.g., "\\t") for 2.0+ don't work
           // when ESCAPED_STRING_LITERALS is enabled. They are parsed literally.
@@ -866,7 +866,7 @@ class ExpressionParserSuite extends AnalysisTest {
   test("composed expressions") {
     assertEqual("1 + r.r As q", (Literal(1) + UnresolvedAttribute("r.r")).as("q"))
     assertEqual("1 - f('o', o(bar))", Literal(1) - 'f.function("o", 'o.function('bar)))
-    intercept("1 - f('o', o(bar)) hello * world", Some("PARSE_INPUT_MISMATCHED"),
+    intercept("1 - f('o', o(bar)) hello * world", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near '*'")
   }
 
@@ -886,7 +886,7 @@ class ExpressionParserSuite extends AnalysisTest {
   test("SPARK-17832 function identifier contains backtick") {
     val complexName = FunctionIdentifier("`ba`r", Some("`fo`o"))
     assertEqual(complexName.quotedString, UnresolvedAttribute(Seq("`fo`o", "`ba`r")))
-    intercept(complexName.unquotedString, Some("PARSE_INPUT_MISMATCHED"),
+    intercept(complexName.unquotedString, Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near")
     // Function identifier contains continuous backticks should be treated correctly.
     val complexName2 = FunctionIdentifier("ba``r", Some("fo``o"))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
index 70138a3..3e2d917 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala
@@ -283,8 +283,8 @@ class PlanParserSuite extends AnalysisTest {
     assertEqual("from a select distinct b, c", Distinct(table("a").select('b, 'c)))
 
     // Weird "FROM table" queries, should be invalid anyway
-    intercept("from a", "no viable alternative at input 'from a'")
-    intercept("from (from a union all from b) c select *", "no viable alternative at input 'from")
+    intercept("from a", "Syntax error at or near end of input")
+    intercept("from (from a union all from b) c select *", "Syntax error at or near 'union'")
   }
 
   test("multi select query") {
@@ -292,10 +292,10 @@ class PlanParserSuite extends AnalysisTest {
       "from a select * select * where s < 10",
       table("a").select(star()).union(table("a").where('s < 10).select(star())))
     intercept(
-      "from a select * select * from x where a.s < 10", Some("PARSE_INPUT_MISMATCHED"),
+      "from a select * select * from x where a.s < 10", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near 'from'")
     intercept(
-      "from a select * from b", Some("PARSE_INPUT_MISMATCHED"),
+      "from a select * from b", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near 'from'")
     assertEqual(
       "from a insert into tbl1 select * insert into tbl2 select * where s < 10",
@@ -404,7 +404,7 @@ class PlanParserSuite extends AnalysisTest {
     val m = intercept[ParseException] {
       parsePlan("SELECT a, b, count(distinct a, distinct b) as c FROM d GROUP BY a, b")
     }.getMessage
-    assert(m.contains("extraneous input 'b'"))
+    assert(m.contains("Syntax error at or near 'b': extra input 'b'"))
 
   }
 
@@ -778,11 +778,11 @@ class PlanParserSuite extends AnalysisTest {
 
   test("select hint syntax") {
     // Hive compatibility: Missing parameter raises ParseException.
-    intercept("SELECT /*+ HINT() */ * FROM t", Some("PARSE_INPUT_MISMATCHED"),
+    intercept("SELECT /*+ HINT() */ * FROM t", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near")
 
     // Disallow space as the delimiter.
-    intercept("SELECT /*+ INDEX(a b c) */ * from default.t", Some("PARSE_INPUT_MISMATCHED"),
+    intercept("SELECT /*+ INDEX(a b c) */ * from default.t", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near 'b'")
 
     comparePlans(
@@ -840,7 +840,7 @@ class PlanParserSuite extends AnalysisTest {
         UnresolvedHint("REPARTITION", Seq(Literal(100)),
           table("t").select(star()))))
 
-    intercept("SELECT /*+ COALESCE(30 + 50) */ * FROM t", Some("PARSE_INPUT_MISMATCHED"),
+    intercept("SELECT /*+ COALESCE(30 + 50) */ * FROM t", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near")
 
     comparePlans(
@@ -965,9 +965,9 @@ class PlanParserSuite extends AnalysisTest {
       )
     }
 
-    intercept("select ltrim(both 'S' from 'SS abc S'", Some("PARSE_INPUT_MISMATCHED"),
+    intercept("select ltrim(both 'S' from 'SS abc S'", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near 'from'") // expecting {')'
-    intercept("select rtrim(trailing 'S' from 'SS abc S'", Some("PARSE_INPUT_MISMATCHED"),
+    intercept("select rtrim(trailing 'S' from 'SS abc S'", Some("PARSE_SYNTAX_ERROR"),
       "Syntax error at or near 'from'") //  expecting {')'
 
     assertTrimPlans(
@@ -1113,7 +1113,7 @@ class PlanParserSuite extends AnalysisTest {
     val m1 = intercept[ParseException] {
       parsePlan("SELECT * FROM (INSERT INTO BAR VALUES (2))")
     }.getMessage
-    assert(m1.contains("missing ')' at 'BAR'"))
+    assert(m1.contains("Syntax error at or near 'BAR': missing ')'"))
     val m2 = intercept[ParseException] {
       parsePlan("SELECT * FROM S WHERE C1 IN (INSERT INTO T VALUES (2))")
     }.getMessage
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala
index a65f209..c2b240b 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/TableIdentifierParserSuite.scala
@@ -310,7 +310,7 @@ class TableIdentifierParserSuite extends SQLKeywordUtils {
         val errMsg = intercept[ParseException] {
           parseTableIdentifier(keyword)
         }.getMessage
-        assert(errMsg.contains("no viable alternative at input"))
+        assert(errMsg.contains("Syntax error at or near"))
         assert(TableIdentifier(keyword) === parseTableIdentifier(s"`$keyword`"))
         assert(TableIdentifier(keyword, Option("db")) === parseTableIdentifier(s"db.`$keyword`"))
       }
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
index cfc77aa..2f46111 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out
@@ -1343,7 +1343,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-extraneous input 'day' expecting {<EOF>, ';'}(line 1, pos 27)
+Syntax error at or near 'day': extra input 'day'(line 1, pos 27)
 
 == SQL ==
 select interval 30 day day day
@@ -1375,7 +1375,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-extraneous input 'days' expecting {<EOF>, ';'}(line 1, pos 29)
+Syntax error at or near 'days': extra input 'days'(line 1, pos 29)
 
 == SQL ==
 select interval 30 days days days
diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out
index b96baee..f13542d 100644
--- a/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/ansi/literals.sql.out
@@ -217,7 +217,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-no viable alternative at input 'select .'(line 1, pos 7)
+Syntax error at or near '.'(line 1, pos 7)
 
 == SQL ==
 select .e3
diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
index 53cae3f..92b454b 100644
--- a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out
@@ -34,7 +34,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 Cannot parse the data type: 
-extraneous input 'InvalidType' expecting <EOF>(line 1, pos 2)
+Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2)
 
 == SQL ==
 a InvalidType
diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out
index b8f6665..6b57246 100644
--- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out
@@ -133,7 +133,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-no viable alternative at input 'WITH t()'(line 1, pos 7)
+Syntax error at or near ')'(line 1, pos 7)
 
 == SQL ==
 WITH t() AS (SELECT 1)
diff --git a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out
index 6af8e70..21c13af 100644
--- a/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/grouping_set.sql.out
@@ -138,7 +138,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-extraneous input 'ROLLUP' expecting {<EOF>, ';'}(line 1, pos 53)
+Syntax error at or near 'ROLLUP': extra input 'ROLLUP'(line 1, pos 53)
 
 == SQL ==
 SELECT a, b, c, count(d) FROM grouping GROUP BY WITH ROLLUP
@@ -152,7 +152,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-extraneous input 'CUBE' expecting {<EOF>, ';'}(line 1, pos 53)
+Syntax error at or near 'CUBE': extra input 'CUBE'(line 1, pos 53)
 
 == SQL ==
 SELECT a, b, c, count(d) FROM grouping GROUP BY WITH CUBE
diff --git a/sql/core/src/test/resources/sql-tests/results/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/interval.sql.out
index f32836e..7aa0c69 100644
--- a/sql/core/src/test/resources/sql-tests/results/interval.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/interval.sql.out
@@ -1336,7 +1336,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-extraneous input 'day' expecting {<EOF>, ';'}(line 1, pos 27)
+Syntax error at or near 'day': extra input 'day'(line 1, pos 27)
 
 == SQL ==
 select interval 30 day day day
@@ -1368,7 +1368,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-extraneous input 'days' expecting {<EOF>, ';'}(line 1, pos 29)
+Syntax error at or near 'days': extra input 'days'(line 1, pos 29)
 
 == SQL ==
 select interval 30 days days days
diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
index 8461083..48a7b7b 100644
--- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out
@@ -125,7 +125,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.AnalysisException
 Cannot parse the data type: 
-extraneous input 'InvalidType' expecting <EOF>(line 1, pos 2)
+Syntax error at or near 'InvalidType': extra input 'InvalidType'(line 1, pos 2)
 
 == SQL ==
 a InvalidType
diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out
index b96baee..f13542d 100644
--- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out
@@ -217,7 +217,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-no viable alternative at input 'select .'(line 1, pos 7)
+Syntax error at or near '.'(line 1, pos 7)
 
 == SQL ==
 select .e3
diff --git a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out
index fc19471..680c570 100644
--- a/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/postgreSQL/window_part3.sql.out
@@ -361,7 +361,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-extraneous input 'BY' expecting {')', ',', 'ORDER', 'RANGE', 'ROWS', 'SORT'}(line 1, pos 45)
+Syntax error at or near 'BY': extra input 'BY'(line 1, pos 45)
 
 == SQL ==
 select rank() OVER (PARTITION BY four, ORDER BY ten) FROM tenk1
diff --git a/sql/core/src/test/resources/sql-tests/results/transform.sql.out b/sql/core/src/test/resources/sql-tests/results/transform.sql.out
index c9a04c9..be57390 100644
--- a/sql/core/src/test/resources/sql-tests/results/transform.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/transform.sql.out
@@ -760,7 +760,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-no viable alternative at input 'SELECT TRANSFORM(b AS'(line 1, pos 19)
+Syntax error at or near 'AS'(line 1, pos 19)
 
 == SQL ==
 SELECT TRANSFORM(b AS b_1, MAX(a), CAST(sum(c) AS STRING))
@@ -782,7 +782,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-no viable alternative at input 'SELECT TRANSFORM(b b_1'(line 1, pos 19)
+Syntax error at or near 'b_1'(line 1, pos 19)
 
 == SQL ==
 SELECT TRANSFORM(b b_1, MAX(a), CAST(sum(c) AS STRING))
@@ -804,7 +804,7 @@ struct<>
 -- !query output
 org.apache.spark.sql.catalyst.parser.ParseException
 
-no viable alternative at input 'SELECT TRANSFORM(b, MAX(a) AS'(line 1, pos 27)
+Syntax error at or near 'AS'(line 1, pos 27)
 
 == SQL ==
 SELECT TRANSFORM(b, MAX(a) AS max_a, CAST(sum(c) AS STRING))
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala
index ebc1bd3..3943922 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/AlterTableRecoverPartitionsParserSuite.scala
@@ -29,7 +29,7 @@ class AlterTableRecoverPartitionsParserSuite extends AnalysisTest with SharedSpa
     val errMsg = intercept[ParseException] {
       parsePlan("ALTER TABLE RECOVER PARTITIONS")
     }.getMessage
-    assert(errMsg.contains("no viable alternative at input 'ALTER TABLE RECOVER PARTITIONS'"))
+    assert(errMsg.contains("Syntax error at or near 'PARTITIONS'"))
   }
 
   test("recover partitions of a table") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
index 5399f96..6cfdbdd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala
@@ -1750,7 +1750,7 @@ class PlanResolutionSuite extends AnalysisTest {
 
     interceptParseException(parsePlan)(
       "CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING)",
-      "extraneous input ':'")()
+      "Syntax error at or near ':': extra input ':'")()
   }
 
   test("create hive table - table file format") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
index 79952e5..1a3e491 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCWriteSuite.scala
@@ -497,7 +497,7 @@ class JDBCWriteSuite extends SharedSparkSession with BeforeAndAfter {
         .option("createTableColumnTypes", "`name char(20)") // incorrectly quoted column
         .jdbc(url1, "TEST.USERDBTYPETEST", properties)
     }.getMessage()
-    assert(msg.contains("extraneous input"))
+    assert(msg.contains("Syntax error at or near '`': extra input '`'"))
   }
 
   test("SPARK-10849: jdbc CreateTableColumnTypes duplicate columns") {
diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
index 2f0fd85..e1840d8 100644
--- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
+++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala
@@ -642,7 +642,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging {
 
   test("SPARK-37694: delete [jar|file|archive] shall use spark sql processor") {
     runCliWithin(2.minute, errorResponses = Seq("ParseException"))(
-      "delete jar dummy.jar;" -> "missing 'FROM' at 'jar'(line 1, pos 7)")
+      "delete jar dummy.jar;" -> "Syntax error at or near 'jar': missing 'FROM'(line 1, pos 7)")
   }
 
   test("SPARK-37906: Spark SQL CLI should not pass final comment") {

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