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

dongjoon 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 3013d8b4d310 [SPARK-45857][SQL] Enforce the error classes in 
sub-classes of `AnalysisException`
3013d8b4d310 is described below

commit 3013d8b4d310997c8a6f6021f860851cd4f3c32a
Author: Max Gekk <max.g...@gmail.com>
AuthorDate: Sun Nov 12 14:56:13 2023 -0800

    [SPARK-45857][SQL] Enforce the error classes in sub-classes of 
`AnalysisException`
    
    ### What changes were proposed in this pull request?
    In the PR, I propose to enforce creation of `AnalysisException` sub-class 
exceptions with an error class always. In particular, it converts the 
constructor with a message to private one, so, callers have to create a 
sub-class of `AnalysisException` 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 `AnalysisException` and its sub-classes 
in regular cases.
    
    ### How was this patch tested?
    By existing test suites, for instance:
    ```
    $ build/sbt "sql/testOnly *QueryParsingErrorsSuite"
    ```
    
    ### Was this patch authored or co-authored using generative AI tooling?
    No.
    
    Closes #43731 from MaxGekk/ban-message-subclasses-AnalysisException.
    
    Authored-by: Max Gekk <max.g...@gmail.com>
    Signed-off-by: Dongjoon Hyun <dh...@apple.com>
---
 .../src/main/resources/error/error-classes.json    |  5 ++
 .../connect/client/GrpcExceptionConverter.scala    | 32 +++------
 .../sql/catalyst/analysis/NonEmptyException.scala  |  2 +-
 .../catalyst/analysis/alreadyExistException.scala  | 45 ++----------
 .../catalyst/analysis/noSuchItemsExceptions.scala  | 79 +++++++---------------
 .../catalyst/analysis/AlreadyExistException.scala  | 14 ----
 .../catalyst/analysis/NoSuchItemException.scala    | 14 ----
 .../catalog/InvalidUDFClassException.scala         |  2 +-
 .../spark/sql/errors/QueryCompilationErrors.scala  |  6 ++
 .../org/apache/spark/sql/jdbc/H2Dialect.scala      |  6 +-
 .../spark/sql/hive/HiveSessionStateBuilder.scala   |  4 +-
 11 files changed, 60 insertions(+), 149 deletions(-)

diff --git a/common/utils/src/main/resources/error/error-classes.json 
b/common/utils/src/main/resources/error/error-classes.json
index 3b7a3a6006ef..e3b9f3161b24 100644
--- a/common/utils/src/main/resources/error/error-classes.json
+++ b/common/utils/src/main/resources/error/error-classes.json
@@ -6334,6 +6334,11 @@
       "Operation not allowed: <cmd> only works on table with location 
provided: <tableIdentWithDB>"
     ]
   },
+  "_LEGACY_ERROR_TEMP_2450" : {
+    "message" : [
+      "No handler for UDF/UDAF/UDTF '<clazz>'"
+    ]
+  },
   "_LEGACY_ERROR_TEMP_3000" : {
     "message" : [
       "Unexpected Py4J server <class>."
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 52bd276b0c4b..73e2db2f4ac7 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
@@ -204,34 +204,24 @@ private[client] object GrpcExceptionConverter {
         messageParameters = params.messageParameters,
         context = params.queryContext)),
     errorConstructor(params =>
-      new NamespaceAlreadyExistsException(
-        params.message,
-        params.errorClass,
-        params.messageParameters)),
+      new NamespaceAlreadyExistsException(params.errorClass.orNull, 
params.messageParameters)),
     errorConstructor(params =>
       new TableAlreadyExistsException(
-        params.message,
-        params.cause,
-        params.errorClass,
-        params.messageParameters)),
+        params.errorClass.orNull,
+        params.messageParameters,
+        params.cause)),
     errorConstructor(params =>
       new TempTableAlreadyExistsException(
-        params.message,
-        params.cause,
-        params.errorClass,
-        params.messageParameters)),
+        params.errorClass.orNull,
+        params.messageParameters,
+        params.cause)),
     errorConstructor(params =>
       new NoSuchDatabaseException(
-        params.message,
-        params.cause,
-        params.errorClass,
-        params.messageParameters)),
+        params.errorClass.orNull,
+        params.messageParameters,
+        params.cause)),
     errorConstructor(params =>
-      new NoSuchTableException(
-        params.message,
-        params.cause,
-        params.errorClass,
-        params.messageParameters)),
+      new NoSuchTableException(params.errorClass.orNull, 
params.messageParameters, params.cause)),
     errorConstructor[NumberFormatException](params =>
       new SparkNumberFormatException(
         params.message,
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
index ecd57672b616..2aea9bac12fe 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/NonEmptyException.scala
@@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.QuotingUtils.quoted
  * Thrown by a catalog when an item already exists. The analyzer will rethrow 
the exception
  * as an [[org.apache.spark.sql.AnalysisException]] with the correct position 
information.
  */
-case class NonEmptyNamespaceException(
+case class NonEmptyNamespaceException private(
     override val message: String,
     override val cause: Option[Throwable] = None)
   extends AnalysisException(message, cause = cause) {
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala
index 85eba2b24614..8932a0296428 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/alreadyExistException.scala
@@ -32,7 +32,7 @@ class DatabaseAlreadyExistsException(db: String)
   extends NamespaceAlreadyExistsException(Array(db))
 
 // any changes to this class should be backward compatible as it may be used 
by external connectors
-class NamespaceAlreadyExistsException private[sql](
+class NamespaceAlreadyExistsException private(
     message: String,
     errorClass: Option[String],
     messageParameters: Map[String, String])
@@ -52,17 +52,10 @@ class NamespaceAlreadyExistsException private[sql](
     this(errorClass = "SCHEMA_ALREADY_EXISTS",
       Map("schemaName" -> quoteNameParts(namespace.toImmutableArraySeq)))
   }
-
-  def this(message: String) = {
-    this(
-      message,
-      errorClass = Some("SCHEMA_ALREADY_EXISTS"),
-      messageParameters = Map.empty[String, String])
-  }
 }
 
 // any changes to this class should be backward compatible as it may be used 
by external connectors
-class TableAlreadyExistsException private[sql](
+class TableAlreadyExistsException private(
     message: String,
     cause: Option[Throwable],
     errorClass: Option[String],
@@ -106,21 +99,13 @@ class TableAlreadyExistsException private[sql](
       messageParameters = Map("relationName" -> quoted(tableIdent)),
       cause = None)
   }
-
-  def this(message: String, cause: Option[Throwable] = None) = {
-    this(
-      message,
-      cause,
-      errorClass = Some("TABLE_OR_VIEW_ALREADY_EXISTS"),
-      messageParameters = Map.empty[String, String])
-  }
 }
 
-class TempTableAlreadyExistsException private[sql](
-  message: String,
-  cause: Option[Throwable],
-  errorClass: Option[String],
-  messageParameters: Map[String, String])
+class TempTableAlreadyExistsException private(
+    message: String,
+    cause: Option[Throwable],
+    errorClass: Option[String],
+    messageParameters: Map[String, String])
   extends AnalysisException(
     message,
     cause = cause,
@@ -144,14 +129,6 @@ class TempTableAlreadyExistsException private[sql](
       messageParameters = Map("relationName"
         -> quoteNameParts(AttributeNameParser.parseAttributeName(table))))
   }
-
-  def this(message: String, cause: Option[Throwable]) = {
-    this(
-      message,
-      cause,
-      errorClass = Some("TEMP_TABLE_OR_VIEW_ALREADY_EXISTS"),
-      messageParameters = Map.empty[String, String])
-  }
 }
 
 // any changes to this class should be backward compatible as it may be used 
by external connectors
@@ -203,12 +180,4 @@ class IndexAlreadyExistsException private(
   def this(indexName: String, tableName: String, cause: Option[Throwable]) = {
     this("INDEX_ALREADY_EXISTS", Map("indexName" -> indexName, "tableName" -> 
tableName), cause)
   }
-
-  def this(message: String, cause: Option[Throwable] = None) = {
-    this(
-      message,
-      cause,
-      errorClass = Some("INDEX_ALREADY_EXISTS"),
-      messageParameters = Map.empty[String, String])
-  }
 }
diff --git 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala
 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala
index b7c8473c08c0..ac22d26ccfd1 100644
--- 
a/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala
+++ 
b/sql/api/src/main/scala/org/apache/spark/sql/catalyst/analysis/noSuchItemsExceptions.scala
@@ -27,21 +27,21 @@ import org.apache.spark.util.ArrayImplicits._
  * Thrown by a catalog when an item cannot be found. The analyzer will rethrow 
the exception
  * as an [[org.apache.spark.sql.AnalysisException]] with the correct position 
information.
  */
-class NoSuchDatabaseException private[sql](
-  message: String,
-  cause: Option[Throwable],
-  errorClass: Option[String],
-  messageParameters: Map[String, String])
+class NoSuchDatabaseException private(
+    message: String,
+    cause: Option[Throwable],
+    errorClass: Option[String],
+    messageParameters: Map[String, String])
   extends AnalysisException(
     message,
     cause = cause,
     errorClass = errorClass,
     messageParameters = messageParameters) {
 
-  def this(errorClass: String, messageParameters: Map[String, String]) = {
+  def this(errorClass: String, messageParameters: Map[String, String], cause: 
Option[Throwable]) = {
     this(
       SparkThrowableHelper.getMessage(errorClass, messageParameters),
-      cause = None,
+      cause = cause,
       Some(errorClass),
       messageParameters)
   }
@@ -49,15 +49,8 @@ class NoSuchDatabaseException private[sql](
   def this(db: String) = {
     this(
       errorClass = "SCHEMA_NOT_FOUND",
-      messageParameters = Map("schemaName" -> quoteIdentifier(db)))
-  }
-
-  def this(message: String, cause: Option[Throwable]) = {
-    this(
-      message = message,
-      cause = cause,
-      errorClass = Some("SCHEMA_NOT_FOUND"),
-      messageParameters = Map.empty[String, String])
+      messageParameters = Map("schemaName" -> quoteIdentifier(db)),
+      cause = None)
   }
 }
 
@@ -90,18 +83,10 @@ class NoSuchNamespaceException private(
     this(errorClass = "SCHEMA_NOT_FOUND",
       Map("schemaName" -> quoteNameParts(namespace.toImmutableArraySeq)))
   }
-
-  def this(message: String, cause: Option[Throwable] = None) = {
-    this(
-      message,
-      cause,
-      errorClass = Some("SCHEMA_NOT_FOUND"),
-      messageParameters = Map.empty[String, String])
-  }
 }
 
 // any changes to this class should be backward compatible as it may be used 
by external connectors
-class NoSuchTableException private[sql](
+class NoSuchTableException private(
     message: String,
     cause: Option[Throwable],
     errorClass: Option[String],
@@ -112,36 +97,34 @@ class NoSuchTableException private[sql](
     errorClass = errorClass,
     messageParameters = messageParameters) {
 
-  def this(errorClass: String, messageParameters: Map[String, String]) = {
+  def this(errorClass: String, messageParameters: Map[String, String], cause: 
Option[Throwable]) = {
     this(
       SparkThrowableHelper.getMessage(errorClass, messageParameters),
-      cause = None,
+      cause = cause,
       Some(errorClass),
       messageParameters)
   }
 
   def this(db: String, table: String) = {
-    this(errorClass = "TABLE_OR_VIEW_NOT_FOUND",
+    this(
+      errorClass = "TABLE_OR_VIEW_NOT_FOUND",
       messageParameters = Map("relationName" ->
-        (quoteIdentifier(db) + "." + quoteIdentifier(table))))
+        (quoteIdentifier(db) + "." + quoteIdentifier(table))),
+      cause = None)
   }
 
   def this(name : Seq[String]) = {
-    this(errorClass = "TABLE_OR_VIEW_NOT_FOUND",
-      messageParameters = Map("relationName" -> quoteNameParts(name)))
+    this(
+      errorClass = "TABLE_OR_VIEW_NOT_FOUND",
+      messageParameters = Map("relationName" -> quoteNameParts(name)),
+      cause = None)
   }
 
   def this(tableIdent: Identifier) = {
-    this(errorClass = "TABLE_OR_VIEW_NOT_FOUND",
-      messageParameters = Map("relationName" -> quoted(tableIdent)))
-  }
-
-  def this(message: String, cause: Option[Throwable] = None) = {
     this(
-      message,
-      cause,
-      errorClass = Some("TABLE_OR_VIEW_NOT_FOUND"),
-      messageParameters = Map.empty[String, String])
+      errorClass = "TABLE_OR_VIEW_NOT_FOUND",
+      messageParameters = Map("relationName" -> quoted(tableIdent)),
+      cause = None)
   }
 }
 
@@ -186,14 +169,6 @@ class NoSuchFunctionException private(
   def this(identifier: Identifier) = {
     this(errorClass = "ROUTINE_NOT_FOUND", Map("routineName" -> 
quoted(identifier)))
   }
-
-  def this(message: String, cause: Option[Throwable] = None) = {
-    this(
-      message,
-      cause,
-      errorClass = Some("ROUTINE_NOT_FOUND"),
-      messageParameters = Map.empty[String, String])
-  }
 }
 
 class NoSuchTempFunctionException(func: String)
@@ -225,12 +200,4 @@ class NoSuchIndexException private(
   def this(indexName: String, tableName: String, cause: Option[Throwable]) = {
     this("INDEX_NOT_FOUND", Map("indexName" -> indexName, "tableName" -> 
tableName), cause)
   }
-
-  def this(message: String, cause: Option[Throwable] = None) = {
-    this(
-      message,
-      cause,
-      errorClass = Some("INDEX_NOT_FOUND"),
-      messageParameters = Map.empty[String, String])
-  }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
index bbac5ab7db3e..4662f1c6ede6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala
@@ -55,13 +55,6 @@ class PartitionAlreadyExistsException private(
         .map( kv => quoteIdentifier(s"${kv._2}") + s" = ${kv._1}").mkString(", 
") + ")"),
         "tableName" -> 
quoteNameParts(UnresolvedAttribute.parseAttributeName(tableName))))
   }
-
-  def this(message: String) = {
-    this(
-      message,
-      errorClass = Some("PARTITIONS_ALREADY_EXIST"),
-      messageParameters = Map.empty[String, String])
-  }
 }
 
 // any changes to this class should be backward compatible as it may be used 
by external connectors
@@ -105,11 +98,4 @@ class PartitionsAlreadyExistException private(
 
   def this(tableName: String, partitionIdent: InternalRow, partitionSchema: 
StructType) =
     this(tableName, Seq(partitionIdent), partitionSchema)
-
-  def this(message: String) = {
-    this(
-      message,
-      errorClass = Some("PARTITIONS_ALREADY_EXIST"),
-      messageParameters = Map.empty[String, String])
-  }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
index 217c293900ec..5db713066ff9 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala
@@ -56,13 +56,6 @@ class NoSuchPartitionException private(
         .map( kv => quoteIdentifier(s"${kv._2}") + s" = ${kv._1}").mkString(", 
") + ")"),
         "tableName" -> 
quoteNameParts(UnresolvedAttribute.parseAttributeName(tableName))))
   }
-
-  def this(message: String) = {
-    this(
-      message,
-      errorClass = Some("PARTITIONS_NOT_FOUND"),
-      messageParameters = Map.empty[String, String])
-  }
 }
 
 // any changes to this class should be backward compatible as it may be used 
by external connectors
@@ -98,11 +91,4 @@ class NoSuchPartitionsException private(
           .mkString(", ")).mkString("), PARTITION (") + ")"),
         "tableName" -> 
quoteNameParts(UnresolvedAttribute.parseAttributeName(tableName))))
   }
-
-  def this(message: String) = {
-    this(
-      message,
-      errorClass = Some("PARTITIONS_NOT_FOUND"),
-      messageParameters = Map.empty[String, String])
-  }
 }
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala
index 658ddb21c6d9..bfd8ba7d5a59 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/InvalidUDFClassException.scala
@@ -24,7 +24,7 @@ import org.apache.spark.sql.AnalysisException
  * Thrown when a query failed for invalid function class, usually because a SQL
  * function's class does not follow the rules of the UDF/UDAF/UDTF class 
definition.
  */
-class InvalidUDFClassException private[sql](
+class InvalidUDFClassException private(
     message: String,
     errorClass: Option[String] = None,
     messageParameters: Map[String, String] = Map.empty)
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
index 7399f6c621cc..c3249a4c02d8 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryCompilationErrors.scala
@@ -3825,4 +3825,10 @@ private[sql] object QueryCompilationErrors extends 
QueryErrorsBase with Compilat
       messageParameters = Map("rowTag" -> toSQLId(optionName))
     )
   }
+
+  def invalidUDFClassError(invalidClass: String): Throwable = {
+    new InvalidUDFClassException(
+      errorClass = "_LEGACY_ERROR_TEMP_2450",
+      messageParameters = Map("invalidClass" -> invalidClass))
+  }
 }
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala 
b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
index 43888d0ffedd..9bed6a6f873e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/H2Dialect.scala
@@ -197,8 +197,10 @@ private[sql] object H2Dialect extends JdbcDialect {
           // TABLE_OR_VIEW_NOT_FOUND_1
           case 42102 =>
             val quotedName = 
quoteNameParts(UnresolvedAttribute.parseAttributeName(message))
-            throw new NoSuchTableException(errorClass = 
"TABLE_OR_VIEW_NOT_FOUND",
-              messageParameters = Map("relationName" -> quotedName))
+            throw new NoSuchTableException(
+              errorClass = "TABLE_OR_VIEW_NOT_FOUND",
+              messageParameters = Map("relationName" -> quotedName),
+              cause = Some(e))
           // SCHEMA_NOT_FOUND_1
           case 90079 =>
             val regex = """"((?:[^"\\]|\\[\\"ntbrf])+)"""".r
diff --git 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
index 0b5e98d0a3e4..e991665e2887 100644
--- 
a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
+++ 
b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala
@@ -30,6 +30,7 @@ import 
org.apache.spark.sql.catalyst.catalog.{ExternalCatalogWithListener, Inval
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.execution.SparkPlanner
 import org.apache.spark.sql.execution.aggregate.ResolveEncodersInScalaAgg
 import org.apache.spark.sql.execution.analysis.DetectAmbiguousSelfJoin
@@ -207,8 +208,7 @@ object HiveUDFExpressionBuilder extends 
SparkUDFExpressionBuilder {
         throw analysisException
     }
     udfExpr.getOrElse {
-      throw new InvalidUDFClassException(
-        s"No handler for UDF/UDAF/UDTF '${clazz.getCanonicalName}'")
+      throw QueryCompilationErrors.invalidUDFClassError(clazz.getCanonicalName)
     }
   }
 }


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

Reply via email to