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/03/02 06:01:42 UTC

[GitHub] [spark] anchovYu opened a new pull request #35707: [WIP][SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

anchovYu opened a new pull request #35707:
URL: https://github.com/apache/spark/pull/35707


   ### What changes were proposed in this pull request?
   This PR handles the case 1 mentioned in https://issues.apache.org/jira/browse/SPARK-38385:
   * Before
       ```
       ParseException: 
       mismatched input 'sel' expecting {'(', 'APPLY', 'CONVERT', 'COPY', 'OPTIMIZE', 'RESTORE', 'ADD', 'ALTER', 'ANALYZE', 'CACHE', 'CLEAR', 'COMMENT', 'COMMIT', 'CREATE', 'DELETE', 'DESC', 'DESCRIBE', 'DFS', 'DROP', 'EXPLAIN', 'EXPORT', 'FROM', 'GRANT', 'IMPORT', 'INSERT', 'LIST', 'LOAD', 'LOCK', 'MAP', 'MERGE', 'MSCK', 'REDUCE', 'REFRESH', 'REPLACE', 'RESET', 'REVOKE', 'ROLLBACK', 'SELECT', 'SET', 'SHOW', 'START', 'SYNC', 'TABLE', 'TRUNCATE', 'UNCACHE', 'UNLOCK', 'UPDATE', 'USE', 'VALUES', 'WITH'}(line 1, pos 0)
       
       == SQL ==
       sel 1
       ^^^ 
       ```
   * After
       ```
       ParseException: 
       syntax error at or near 'sel'(line 1, pos 0)
       
       == SQL ==
       sel 1
       ^^^ 
       ```
   
   #### Implementation general idea
   ANTLR uses the DefaultErrorStrategy class to create error messages: 
   
   ```scala
   public class DefaultErrorStrategy implements ANTLRErrorStrategy {
     protected void reportInputMismatch(Parser recognizer, InputMismatchException e)
     {
   	String msg = "mismatched input " + 
                       getTokenErrorDisplay(e.getOffendingToken()) + " expecting " +
                       e.getExpectedTokens().toString(recognizer.getVocabulary());
   	recognizer.notifyErrorListeners(e.getOffendingToken(), msg, e);
     }
     ..
   }
   ```
   It is easy to extend the `DefaultErrorStrategy` and override corresponding functions to output better error messages. Then in our parser, set the error strategy to be the one we created.
   
   #### Changes in code
   To achieve this, the following changes are made:
   * error-classes.json
       Define a new type of error `PARSE_INPUT_MISMATCHED` with the new error framework:
       ```json
         "PARSE_INPUT_MISMATCHED" : {
           "message" : [ "syntax error at or near %s" ],
           "sqlState" : "42000"
         },
       ```
   * SparkParserErrorStrategy.scala
       This is a new class, extending the `org.antlr.v4.runtime.DefaultErrorStrategy` that does special handling on errors. Note the original `DefaultErrorStrategy` is where the `mismatched input` error message generates from. 
       The new class is intended to provide more information, e.g. the error class and the message parameters, on these errors encountered in ANTLR parser to the downstream consumers to be able to apply the `SparkThrowable` error message framework to these exceptions.
   
   * ParserDriver.scala
     * It sets the error strategy of the parser to be the above new `SparkParserErrorStrategy`.
     * When catching an exception thrown from ANTLR, when it can find out the error class and message parameter, it creates `ParseException` with this information, which composes the error message through `SparkThrowableHelper.getMessage`. It then formalizes the standard error messages of these types.
   
   * test suites
       It change all affected test suites. It also adds a check on the error class, note the newly added `PARSE_INPUT_MISMATCHED` in the after case:
       ```scala
       // before
       intercept("select * from r order by q from t", 1, 27, 31,
         "mismatched input",
         "---------------------------^^^")
       // after
       intercept("select * from r order by q from t", "PARSE_INPUT_MISMATCHED",
         1, 27, 31,
         "syntax error at or near",
         "---------------------------^^^"
       )
       ```
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description and/or an example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Spark versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   If benchmark tests were added, please run the benchmarks in GitHub Actions for the consistent environment, and the instructions could accord to: https://spark.apache.org/developer-tools.html#github-workflow-benchmarks.
   -->
   


-- 
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] AmplabJenkins commented on pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
AmplabJenkins commented on pull request #35707:
URL: https://github.com/apache/spark/pull/35707#issuecomment-1057615965


   Can one of the admins verify this patch?


-- 
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] anchovYu commented on pull request #35707: [WIP][SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on pull request #35707:
URL: https://github.com/apache/spark/pull/35707#issuecomment-1056346664


   @MaxGekk @cloud-fan would you take a look? Thanks!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan edited a comment on pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan edited a comment on pull request #35707:
URL: https://github.com/apache/spark/pull/35707#issuecomment-1060295860


   Yea we don't have a very good style detection script. In practice we follow https://github.com/databricks/scala-style-guide


-- 
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] anchovYu commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r821183467



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -200,11 +200,15 @@ trait AnalysisTest extends PlanTest {
     }
   }
 
-  protected def interceptParseException(
-      parser: String => Any)(sqlCommand: String, messages: String*): Unit = {
+  protected def interceptParseException(parser: String => Any)(
+    sqlCommand: String, messages: String*)(
+    errorClass: Option[String] = None): Unit = {

Review comment:
       The intention is to provide default value `None` for the errorClass and potential future added fields like sqlState. Becasue message is a *-parameter, the `errorClass` with default value can't be placed before it. Especcially if new fields are added, this interface can simply be extended to 
   ```scala
     protected def interceptParseException(parser: String => Any)(
       sqlCommand: String, messages: String*)(
       errorClass: Option[String] = None, newField: Option[String] = None):
   ```
   and the caller doesn't need to change its signature. For example, 
   ``` scala
     private def intercept(sqlCommand: String, messages: String*): Unit =
       interceptParseException(parsePlan)(sqlCommand, messages: _*)()
   
     private def intercept(sqlCommand: String, errorClass: Option[String], messages: String*): Unit =
       interceptParseException(parsePlan)(sqlCommand, messages: _*)(errorClass)
   ```
   will still compile.




-- 
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] anchovYu commented on pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on pull request #35707:
URL: https://github.com/apache/spark/pull/35707#issuecomment-1058558335


   Thanks @cloud-fan for the review. I feel sorry for all the linting problem.. Before I commit, I ran the dev/scalastyle and it shows all style has passed, then I thought it is fine. So to avoid this happen again next time and to confirm, this scalastyle check is not enough right? We should still follow the code guidance manually?


-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r820466576



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -200,11 +200,15 @@ trait AnalysisTest extends PlanTest {
     }
   }
 
-  protected def interceptParseException(
-      parser: String => Any)(sqlCommand: String, messages: String*): Unit = {
+  protected def interceptParseException(parser: String => Any)(
+    sqlCommand: String, messages: String*)(
+    errorClass: Option[String] = None): Unit = {

Review comment:
       nit: in other places, we just add the `errorClass` parameter before the `messages` parameter, e.g. `def intercept(sqlCommand: String, errorClass: Option[String], messages: String*)`, but why do we add a curry constructor here?




-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r818308402



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
##########
@@ -1774,7 +1777,7 @@ class DDLParserSuite extends AnalysisTest {
         allColumns = true))
 
     intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value",
-      "mismatched input 'key' expecting {<EOF>, ';'}")
+      Some("PARSE_INPUT_MISMATCHED"), "syntax error at or near 'key'") // expecting {<EOF>, ';'}

Review comment:
       `expecting {<EOF>, ';'}` is not in the error message any more.




-- 
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] anchovYu commented on a change in pull request #35707: [WIP][SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r817973758



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.parser
+
+import scala.util.{Failure, Success, Try}
+
+import org.antlr.v4.runtime.{DefaultErrorStrategy, InputMismatchException, IntStream, Parser,
+  ParserRuleContext, RecognitionException, Recognizer}
+
+/**
+ * A [[SparkRecognitionException]] extends the [[RecognitionException]] with more information
+ * including the error class and parameters for the error message, which align with the interface
+ * of [[SparkThrowableHelper]].
+ */
+private[parser] class SparkRecognitionException(
+    message: String,
+    recognizer: Recognizer[_, _],
+    input: IntStream,
+    ctx: ParserRuleContext,
+    val errorClass: Option[String] = None,

Review comment:
       Yes, the `PARSE_INPUT_MISMATCHED` is the first step. I have the plan to improve some other error messages with new error class attached.




-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r818307466



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.parser
+
+import scala.util.{Failure, Success, Try}
+
+import org.antlr.v4.runtime.{DefaultErrorStrategy, InputMismatchException, IntStream, Parser,
+  ParserRuleContext, RecognitionException, Recognizer}
+
+/**
+ * A [[SparkRecognitionException]] extends the [[RecognitionException]] with more information
+ * including the error class and parameters for the error message, which align with the interface
+ * of [[SparkThrowableHelper]].
+ */
+private[parser] class SparkRecognitionException(
+    message: String,
+    recognizer: Recognizer[_, _],
+    input: IntStream,
+    ctx: ParserRuleContext,
+    val errorClass: Option[String] = None,
+    val messageParameters: Array[String] = Array.empty)
+  extends RecognitionException(message, recognizer, input, ctx) {
+
+  /** Construct from a given [[RecognitionException]], with additional error information. */
+  def this(
+      recognitionException: RecognitionException,
+      errorClass: String,
+      messageParameters: Array[String]) =
+    this(
+      recognitionException.getMessage,
+      recognitionException.getRecognizer,
+      recognitionException.getInputStream,
+      Try { recognitionException.getCtx.asInstanceOf[ParserRuleContext] } match {

Review comment:
       how about
   ```
   recognitionException.getCtx match {
     case p: ParserRuleContext => p
     case _ => null
   }
   ```




-- 
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] anchovYu commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r821183467



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -200,11 +200,15 @@ trait AnalysisTest extends PlanTest {
     }
   }
 
-  protected def interceptParseException(
-      parser: String => Any)(sqlCommand: String, messages: String*): Unit = {
+  protected def interceptParseException(parser: String => Any)(
+    sqlCommand: String, messages: String*)(
+    errorClass: Option[String] = None): Unit = {

Review comment:
       The intention is to provide default value `None` for the errorClass and potential future added fields like sqlState. Especcially if new fields are added, this interface can simply be extended to 
   ```scala
     protected def interceptParseException(parser: String => Any)(
       sqlCommand: String, messages: String*)(
       errorClass: Option[String] = None, newField: Option[String] = None):
   ```
   and the caller doesn't need to change its signature. For example, 
   ``` scala
     private def intercept(sqlCommand: String, messages: String*): Unit =
       interceptParseException(parsePlan)(sqlCommand, messages: _*)()
   
     private def intercept(sqlCommand: String, errorClass: Option[String], messages: String*): Unit =
       interceptParseException(parsePlan)(sqlCommand, messages: _*)(errorClass)
   ```
   will still compile.




-- 
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] anchovYu commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r819095750



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala
##########
@@ -1774,7 +1777,7 @@ class DDLParserSuite extends AnalysisTest {
         allColumns = true))
 
     intercept("ANALYZE TABLE a.b.c COMPUTE STATISTICS FOR ALL COLUMNS key, value",
-      "mismatched input 'key' expecting {<EOF>, ';'}")
+      Some("PARSE_INPUT_MISMATCHED"), "syntax error at or near 'key'") // expecting {<EOF>, ';'}

Review comment:
       I make it a comment in case in the future we are able to provide suggestion, and that may serves a reminder of what are really expected in the pre-changed code.
   But I can remove it anyways.




-- 
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] cloud-fan commented on a change in pull request #35707: [WIP][SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r817729005



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.parser
+
+import scala.util.{Failure, Success, Try}
+
+import org.antlr.v4.runtime.{DefaultErrorStrategy, InputMismatchException, IntStream, Parser,
+  ParserRuleContext, RecognitionException, Recognizer}
+
+/**
+ * A [[SparkRecognitionException]] extends the [[RecognitionException]] with more information
+ * including the error class and parameters for the error message, which align with the interface
+ * of [[SparkThrowableHelper]].
+ */
+private[parser] class SparkRecognitionException(
+    message: String,
+    recognizer: Recognizer[_, _],
+    input: IntStream,
+    ctx: ParserRuleContext,
+    val errorClass: Option[String] = None,

Review comment:
       are we going to pass error classes other than `PARSE_INPUT_MISMATCHED` to it?




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



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


[GitHub] [spark] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r818309104



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
##########
@@ -31,26 +32,47 @@ class ErrorParserSuite extends AnalysisTest {
     assert(parsePlan(sqlCommand) == plan)
   }
 
-  def intercept(sqlCommand: String, messages: String*): Unit =
-    interceptParseException(CatalystSqlParser.parsePlan)(sqlCommand, messages: _*)
-
-  def intercept(sql: String, line: Int, startPosition: Int, stopPosition: Int,
-                messages: String*): Unit = {
+  private def interceptImpl(sql: String, messages: String*)(
+    line: Option[Int] = None, startPosition: Option[Int] = None, stopPosition: Option[Int] = None)(
+    errorClass: Option[String] = None): Unit = {

Review comment:
       ```suggestion
       line: Option[Int] = None,
       startPosition: Option[Int] = None, 
       stopPosition: Option[Int] = None,
       errorClass: Option[String] = None): Unit = {
   ```




-- 
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] anchovYu commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r821183467



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -200,11 +200,15 @@ trait AnalysisTest extends PlanTest {
     }
   }
 
-  protected def interceptParseException(
-      parser: String => Any)(sqlCommand: String, messages: String*): Unit = {
+  protected def interceptParseException(parser: String => Any)(
+    sqlCommand: String, messages: String*)(
+    errorClass: Option[String] = None): Unit = {

Review comment:
       The intention is to provide default value `None` for the errorClass and potential future added fields like sqlState. Becasue `messages` is a *-parameter, the `errorClass` with default value can't be placed before it. Especcially if new fields are added, this interface can simply be extended to 
   ```scala
     protected def interceptParseException(parser: String => Any)(
       sqlCommand: String, messages: String*)(
       errorClass: Option[String] = None, newField: Option[String] = None):
   ```
   and the caller doesn't need to change its signature. For example, 
   ``` scala
     private def intercept(sqlCommand: String, messages: String*): Unit =
       interceptParseException(parsePlan)(sqlCommand, messages: _*)()
   
     private def intercept(sqlCommand: String, errorClass: Option[String], messages: String*): Unit =
       interceptParseException(parsePlan)(sqlCommand, messages: _*)(errorClass)
   ```
   will still compile.




-- 
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] anchovYu commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
anchovYu commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r821184162



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -200,11 +200,15 @@ trait AnalysisTest extends PlanTest {
     }
   }
 
-  protected def interceptParseException(
-      parser: String => Any)(sqlCommand: String, messages: String*): Unit = {
+  protected def interceptParseException(parser: String => Any)(
+    sqlCommand: String, messages: String*)(
+    errorClass: Option[String] = None): Unit = {

Review comment:
       WDYT? Is there another set of designs in the consideration of easy extension?




-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r818307012



##########
File path: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/SparkParserErrorStrategy.scala
##########
@@ -0,0 +1,78 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.catalyst.parser
+
+import scala.util.{Failure, Success, Try}
+
+import org.antlr.v4.runtime.{DefaultErrorStrategy, InputMismatchException, IntStream, Parser,
+  ParserRuleContext, RecognitionException, Recognizer}
+
+/**
+ * A [[SparkRecognitionException]] extends the [[RecognitionException]] with more information
+ * including the error class and parameters for the error message, which align with the interface
+ * of [[SparkThrowableHelper]].
+ */
+private[parser] class SparkRecognitionException(

Review comment:
       ```suggestion
   class SparkRecognitionException(
   ```
   everything in catalyst is private, we don't need to add private modifiers.




-- 
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] cloud-fan closed pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan closed pull request #35707:
URL: https://github.com/apache/spark/pull/35707


   


-- 
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] cloud-fan commented on pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35707:
URL: https://github.com/apache/spark/pull/35707#issuecomment-1060295860


   Yea we don't have a very good style detection script. In practise we follow https://github.com/databricks/scala-style-guide


-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r818309528



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
##########
@@ -31,26 +32,47 @@ class ErrorParserSuite extends AnalysisTest {
     assert(parsePlan(sqlCommand) == plan)
   }
 
-  def intercept(sqlCommand: String, messages: String*): Unit =
-    interceptParseException(CatalystSqlParser.parsePlan)(sqlCommand, messages: _*)
-
-  def intercept(sql: String, line: Int, startPosition: Int, stopPosition: Int,
-                messages: String*): Unit = {
+  private def interceptImpl(sql: String, messages: String*)(
+    line: Option[Int] = None, startPosition: Option[Int] = None, stopPosition: Option[Int] = None)(
+    errorClass: Option[String] = None): Unit = {
     val e = intercept[ParseException](CatalystSqlParser.parsePlan(sql))
 
-    // Check position.
-    assert(e.line.isDefined)
-    assert(e.line.get === line)
-    assert(e.startPosition.isDefined)
-    assert(e.startPosition.get === startPosition)
-    assert(e.stop.startPosition.isDefined)
-    assert(e.stop.startPosition.get === stopPosition)
-
     // Check messages.
     val error = e.getMessage
     messages.foreach { message =>
       assert(error.contains(message))
     }
+
+    // Check position.
+    if (line.isDefined) {
+      assert(line.isDefined && startPosition.isDefined && stopPosition.isDefined)
+      assert(e.line.isDefined)
+      assert(e.line.get === line.get)
+      assert(e.startPosition.isDefined)
+      assert(e.startPosition.get === startPosition.get)
+      assert(e.stop.startPosition.isDefined)
+      assert(e.stop.startPosition.get === stopPosition.get)
+    }
+
+    // Check error class.
+    if (errorClass.isDefined) {
+      assert(e.getErrorClass == errorClass.get)
+    }
+  }
+
+  def intercept(sqlCommand: String, errorClass: Option[String], messages: String*): Unit = {
+      interceptImpl(sqlCommand, messages: _*)()(errorClass)
+  }
+
+  def intercept(sql: String, line: Int, startPosition: Int, stopPosition: Int,
+    messages: String*): Unit = {

Review comment:
       ```suggestion
     def intercept(
         sql: String, line: Int, startPosition: Int, stopPosition: Int, messages: String*): Unit = {
   ```




-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r818309367



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala
##########
@@ -31,26 +32,47 @@ class ErrorParserSuite extends AnalysisTest {
     assert(parsePlan(sqlCommand) == plan)
   }
 
-  def intercept(sqlCommand: String, messages: String*): Unit =
-    interceptParseException(CatalystSqlParser.parsePlan)(sqlCommand, messages: _*)
-
-  def intercept(sql: String, line: Int, startPosition: Int, stopPosition: Int,
-                messages: String*): Unit = {
+  private def interceptImpl(sql: String, messages: String*)(
+    line: Option[Int] = None, startPosition: Option[Int] = None, stopPosition: Option[Int] = None)(
+    errorClass: Option[String] = None): Unit = {
     val e = intercept[ParseException](CatalystSqlParser.parsePlan(sql))
 
-    // Check position.
-    assert(e.line.isDefined)
-    assert(e.line.get === line)
-    assert(e.startPosition.isDefined)
-    assert(e.startPosition.get === startPosition)
-    assert(e.stop.startPosition.isDefined)
-    assert(e.stop.startPosition.get === stopPosition)
-
     // Check messages.
     val error = e.getMessage
     messages.foreach { message =>
       assert(error.contains(message))
     }
+
+    // Check position.
+    if (line.isDefined) {
+      assert(line.isDefined && startPosition.isDefined && stopPosition.isDefined)
+      assert(e.line.isDefined)
+      assert(e.line.get === line.get)
+      assert(e.startPosition.isDefined)
+      assert(e.startPosition.get === startPosition.get)
+      assert(e.stop.startPosition.isDefined)
+      assert(e.stop.startPosition.get === stopPosition.get)
+    }
+
+    // Check error class.
+    if (errorClass.isDefined) {
+      assert(e.getErrorClass == errorClass.get)
+    }
+  }
+
+  def intercept(sqlCommand: String, errorClass: Option[String], messages: String*): Unit = {
+      interceptImpl(sqlCommand, messages: _*)()(errorClass)

Review comment:
       nit: 2 spaces indentation




-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r818305904



##########
File path: core/src/main/resources/error/error-classes.json
##########
@@ -122,6 +122,10 @@
     "message" : [ "PARTITION clause cannot contain a non-partition column name: %s" ],
     "sqlState" : "42000"
   },
+  "PARSE_INPUT_MISMATCHED" : {
+    "message" : [ "syntax error at or near %s" ],

Review comment:
       ```suggestion
       "message" : [ "Syntax error at or near %s" ],
   ```




-- 
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] cloud-fan commented on pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on pull request #35707:
URL: https://github.com/apache/spark/pull/35707#issuecomment-1061311021


   thanks, merging to master!


-- 
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] cloud-fan commented on a change in pull request #35707: [SPARK-38385][SQL] Improve error messages of 'mismatched input' cases from ANTLR

Posted by GitBox <gi...@apache.org>.
cloud-fan commented on a change in pull request #35707:
URL: https://github.com/apache/spark/pull/35707#discussion_r820466576



##########
File path: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
##########
@@ -200,11 +200,15 @@ trait AnalysisTest extends PlanTest {
     }
   }
 
-  protected def interceptParseException(
-      parser: String => Any)(sqlCommand: String, messages: String*): Unit = {
+  protected def interceptParseException(parser: String => Any)(
+    sqlCommand: String, messages: String*)(
+    errorClass: Option[String] = None): Unit = {

Review comment:
       nit: in other places, we just add the `errorClass` parameter before the `messages` parameter, e.g. `def intercept(sqlCommand: String, errorClass: Option[String], messages: String*)`, but why do we add a curried constructor here?




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