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 d582b74d97a7 [SPARK-45824][SQL] Enforce the error class in
`ParseException`
d582b74d97a7 is described below
commit d582b74d97a7e44bdd2f0ae6c63121fc5e5466b7
Author: Max Gekk <[email protected]>
AuthorDate: Wed Nov 8 11:38:55 2023 +0300
[SPARK-45824][SQL] Enforce the error class in `ParseException`
### What changes were proposed in this pull request?
In the PR, I propose to enforce creation of `ParseException` with an error
class always. In particular, it converts the constructor with a message to
private one, so, callers have to create `ParseException` with an error class.
### Why are the changes needed?
This simplifies migration on error classes.
### Does this PR introduce _any_ user-facing change?
No since user code doesn't throw `ParseException` in regular cases.
### How was this patch tested?
By existing test suites, for instance:
```
$ build/sbt "sql/testOnly *QueryParsingErrorsSuite"
$ build/sbt "test:testOnly *SparkConnectClientSuite"
```
### Was this patch authored or co-authored using generative AI tooling?
No.
Closes #43702 from MaxGekk/ban-message-ParseException.
Authored-by: Max Gekk <[email protected]>
Signed-off-by: Max Gekk <[email protected]>
---
.../connect/client/SparkConnectClientSuite.scala | 15 ++----
.../connect/client/GrpcExceptionConverter.scala | 3 +-
.../apache/spark/sql/catalyst/parser/parsers.scala | 53 ++++++++++++++++------
.../spark/sql/errors/QueryParsingErrors.scala | 8 +++-
4 files changed, 51 insertions(+), 28 deletions(-)
diff --git
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
index b3ff4eb0bb29..d0c85da5f212 100644
---
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
+++
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/connect/client/SparkConnectClientSuite.scala
@@ -31,7 +31,6 @@ import org.apache.spark.{SparkException, SparkThrowable}
import org.apache.spark.connect.proto
import org.apache.spark.connect.proto.{AddArtifactsRequest,
AddArtifactsResponse, AnalyzePlanRequest, AnalyzePlanResponse,
ArtifactStatusesRequest, ArtifactStatusesResponse, ExecutePlanRequest,
ExecutePlanResponse, SparkConnectServiceGrpc}
import org.apache.spark.sql.SparkSession
-import org.apache.spark.sql.catalyst.parser.ParseException
import org.apache.spark.sql.connect.common.config.ConnectCommon
import org.apache.spark.sql.test.ConnectFunSuite
@@ -210,19 +209,15 @@ class SparkConnectClientSuite extends ConnectFunSuite
with BeforeAndAfterEach {
}
for ((name, constructor) <- GrpcExceptionConverter.errorFactory) {
- test(s"error framework parameters - ${name}") {
+ test(s"error framework parameters - $name") {
val testParams = GrpcExceptionConverter.ErrorParams(
- message = "test message",
+ message = "Found duplicate keys `abc`",
cause = None,
- errorClass = Some("test error class"),
- messageParameters = Map("key" -> "value"),
+ errorClass = Some("DUPLICATE_KEY"),
+ messageParameters = Map("keyColumn" -> "`abc`"),
queryContext = Array.empty)
val error = constructor(testParams)
- if (!error.isInstanceOf[ParseException]) {
- assert(error.getMessage == testParams.message)
- } else {
- assert(error.getMessage == s"\n${testParams.message}")
- }
+ assert(error.getMessage.contains(testParams.message))
assert(error.getCause == null)
error match {
case sparkThrowable: SparkThrowable =>
diff --git
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
index 652797bc2e40..88cd2118ba75 100644
---
a/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
+++
b/connector/connect/common/src/main/scala/org/apache/spark/sql/connect/client/GrpcExceptionConverter.scala
@@ -191,10 +191,9 @@ private[client] object GrpcExceptionConverter {
errorConstructor(params =>
new ParseException(
None,
- params.message,
Origin(),
Origin(),
- errorClass = params.errorClass,
+ errorClass = params.errorClass.orNull,
messageParameters = params.messageParameters,
queryContext = params.queryContext)),
errorConstructor(params =>
diff --git
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
index 22e6c67090b4..2689e317128a 100644
--- a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
+++ b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/parser/parsers.scala
@@ -23,7 +23,7 @@ import org.antlr.v4.runtime.atn.PredictionMode
import org.antlr.v4.runtime.misc.{Interval, ParseCancellationException}
import org.antlr.v4.runtime.tree.TerminalNodeImpl
-import org.apache.spark.{QueryContext, SparkThrowable, SparkThrowableHelper}
+import org.apache.spark.{QueryContext, SparkException, SparkThrowable,
SparkThrowableHelper}
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, Origin,
SQLQueryContext, WithOrigin}
@@ -99,10 +99,9 @@ abstract class AbstractParser extends
DataTypeParserInterface with Logging {
case e: SparkThrowable with WithOrigin =>
throw new ParseException(
command = Option(command),
- message = e.getMessage,
start = e.origin,
stop = e.origin,
- errorClass = Option(e.getErrorClass),
+ errorClass = e.getErrorClass,
messageParameters = e.getMessageParameters.asScala.toMap,
queryContext = e.getQueryContext)
}
@@ -174,7 +173,12 @@ case object ParseErrorListener extends BaseErrorListener {
case sre: SparkRecognitionException if sre.errorClass.isDefined =>
throw new ParseException(None, start, stop, sre.errorClass.get,
sre.messageParameters)
case _ =>
- throw new ParseException(None, msg, start, stop)
+ throw new ParseException(
+ command = None,
+ start = start,
+ stop = stop,
+ errorClass = "PARSE_SYNTAX_ERROR",
+ messageParameters = Map("error" -> msg, "hint" -> ""))
}
}
}
@@ -183,7 +187,7 @@ case object ParseErrorListener extends BaseErrorListener {
* A [[ParseException]] is an [[SparkException]] that is thrown during the
parse process. It
* contains fields and an extended error message that make reporting and
diagnosing errors easier.
*/
-class ParseException(
+class ParseException private(
val command: Option[String],
message: String,
val start: Origin,
@@ -223,7 +227,24 @@ class ParseException(
start,
stop,
Some(errorClass),
- messageParameters)
+ messageParameters,
+ queryContext = ParseException.getQueryContext())
+
+ def this(
+ command: Option[String],
+ start: Origin,
+ stop: Origin,
+ errorClass: String,
+ messageParameters: Map[String, String],
+ queryContext: Array[QueryContext]) =
+ this(
+ command,
+ SparkThrowableHelper.getMessage(errorClass, messageParameters),
+ start,
+ stop,
+ Some(errorClass),
+ messageParameters,
+ queryContext)
override def getMessage: String = {
val builder = new StringBuilder
@@ -247,17 +268,21 @@ class ParseException(
}
def withCommand(cmd: String): ParseException = {
- val (cls, params) =
- if (errorClass == Some("PARSE_SYNTAX_ERROR") && cmd.trim().isEmpty) {
- // PARSE_EMPTY_STATEMENT error class overrides the PARSE_SYNTAX_ERROR
when cmd is empty
- (Some("PARSE_EMPTY_STATEMENT"), Map.empty[String, String])
- } else {
- (errorClass, messageParameters)
- }
- new ParseException(Option(cmd), message, start, stop, cls, params,
queryContext)
+ val cl = getErrorClass
+ val (newCl, params) = if (cl == "PARSE_SYNTAX_ERROR" &&
cmd.trim().isEmpty) {
+ // PARSE_EMPTY_STATEMENT error class overrides the PARSE_SYNTAX_ERROR
when cmd is empty
+ ("PARSE_EMPTY_STATEMENT", Map.empty[String, String])
+ } else {
+ (cl, messageParameters)
+ }
+ new ParseException(Option(cmd), start, stop, newCl, params, queryContext)
}
override def getQueryContext: Array[QueryContext] = queryContext
+
+ override def getErrorClass: String = errorClass.getOrElse {
+ throw SparkException.internalError("ParseException shall have an error
class.")
+ }
}
object ParseException {
diff --git
a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
index f63fc8c4785b..2067bf7d0955 100644
---
a/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
+++
b/sql/api/src/main/scala/org/apache/spark/sql/errors/QueryParsingErrors.scala
@@ -431,8 +431,12 @@ private[sql] object QueryParsingErrors extends
DataTypeErrorsBase {
}
def sqlStatementUnsupportedError(sqlText: String, position: Origin):
Throwable = {
- new ParseException(Option(sqlText), "Unsupported SQL statement", position,
position,
- Some("_LEGACY_ERROR_TEMP_0039"))
+ new ParseException(
+ command = Option(sqlText),
+ start = position,
+ stop = position,
+ errorClass = "_LEGACY_ERROR_TEMP_0039",
+ messageParameters = Map.empty)
}
def invalidIdentifierError(ident: String, ctx: ErrorIdentContext): Throwable
= {
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]