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

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


The following commit(s) were added to refs/heads/branch-3.5 by this push:
     new b179c0cb22e [SPARK-44785][SQL][CONNECT] Convert common 
alreadyExistsExceptions and noSuchExceptions
b179c0cb22e is described below

commit b179c0cb22e70da67e88b17719d3d22c815530fd
Author: Yihong He <yihong...@databricks.com>
AuthorDate: Mon Aug 21 12:13:53 2023 +0900

    [SPARK-44785][SQL][CONNECT] Convert common alreadyExistsExceptions and 
noSuchExceptions
    
    ### What changes were proposed in this pull request?
    
    - Convert common alreadyExistsExceptions and noSuchExceptions
    - Extend common alreadyExistsExceptions and noSuchExceptions to support 
(message, cause) constructors
    
    ### Why are the changes needed?
    
    - Better compatibility with the existing control flow
    - Better readability of errors
    
    ### Does this PR introduce _any_ user-facing change?
    
    No
    
    ### How was this patch tested?
    
    `build/sbt "connect-client-jvm/testOnly *ClientE2ETestSuite"`
    
    Closes #42471 from heyihong/SPARK-44785.
    
    Authored-by: Yihong He <yihong...@databricks.com>
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
    (cherry picked from commit e73487c4348c47571a3ea083a0903a7997b64a47)
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
---
 .../org/apache/spark/sql/ClientE2ETestSuite.scala  | 44 ++++++++++++++++++++++
 .../connect/client/GrpcExceptionConverter.scala    | 13 +++++--
 .../catalyst/analysis/alreadyExistException.scala  | 36 ++++++++++++++++--
 .../catalyst/analysis/noSuchItemsExceptions.scala  | 36 ++++++++++++++++--
 4 files changed, 119 insertions(+), 10 deletions(-)

diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
index 074cf170dd3..1bfc1cfc7a8 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientE2ETestSuite.scala
@@ -31,6 +31,7 @@ import org.scalatest.PrivateMethodTester
 
 import org.apache.spark.SparkBuildInfo.{spark_version => SPARK_VERSION}
 import org.apache.spark.SparkException
+import 
org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, 
NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException, 
TempTableAlreadyExistsException}
 import org.apache.spark.sql.catalyst.encoders.AgnosticEncoders.StringEncoder
 import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema
 import org.apache.spark.sql.catalyst.parser.ParseException
@@ -43,6 +44,49 @@ import org.apache.spark.sql.types._
 
 class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with 
PrivateMethodTester {
 
+  test("throw NoSuchDatabaseException") {
+    intercept[NoSuchDatabaseException] {
+      spark.sql("use database123")
+    }
+  }
+
+  test("throw NoSuchTableException") {
+    intercept[NoSuchTableException] {
+      spark.catalog.getTable("test_table")
+    }
+  }
+
+  test("throw NamespaceAlreadyExistsException") {
+    try {
+      spark.sql("create database test_db")
+      intercept[NamespaceAlreadyExistsException] {
+        spark.sql("create database test_db")
+      }
+    } finally {
+      spark.sql("drop database test_db")
+    }
+  }
+
+  test("throw TempTableAlreadyExistsException") {
+    try {
+      spark.sql("create temporary view test_view as select 1")
+      intercept[TempTableAlreadyExistsException] {
+        spark.sql("create temporary view test_view as select 1")
+      }
+    } finally {
+      spark.sql("drop view test_view")
+    }
+  }
+
+  test("throw TableAlreadyExistsException") {
+    withTable("testcat.test_table") {
+      spark.sql(s"create table testcat.test_table (id int)")
+      intercept[TableAlreadyExistsException] {
+        spark.sql(s"create table testcat.test_table (id int)")
+      }
+    }
+  }
+
   test("throw ParseException") {
     intercept[ParseException] {
       spark.sql("selet 1").collect()
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 9b8181cb88f..cf92a53847e 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
@@ -25,6 +25,7 @@ import io.grpc.protobuf.StatusProto
 
 import org.apache.spark.SparkException
 import org.apache.spark.sql.AnalysisException
+import 
org.apache.spark.sql.catalyst.analysis.{NamespaceAlreadyExistsException, 
NoSuchDatabaseException, NoSuchTableException, TableAlreadyExistsException, 
TempTableAlreadyExistsException}
 import org.apache.spark.sql.catalyst.parser.ParseException
 import org.apache.spark.sql.catalyst.trees.Origin
 import org.apache.spark.util.JsonUtils
@@ -62,14 +63,20 @@ private[client] object GrpcExceptionConverter extends 
JsonUtils {
   }
 
   private def errorConstructor[T <: Throwable: ClassTag](
-      throwableCtr: (String, Throwable) => T): (String, (String, Throwable) => 
Throwable) = {
+      throwableCtr: (String, Option[Throwable]) => T)
+      : (String, (String, Option[Throwable]) => Throwable) = {
     val className = implicitly[reflect.ClassTag[T]].runtimeClass.getName
     (className, throwableCtr)
   }
 
   private val errorFactory = Map(
     errorConstructor((message, _) => new ParseException(None, message, 
Origin(), Origin())),
-    errorConstructor((message, cause) => new AnalysisException(message, cause 
= Option(cause))))
+    errorConstructor((message, cause) => new AnalysisException(message, cause 
= cause)),
+    errorConstructor((message, _) => new 
NamespaceAlreadyExistsException(message)),
+    errorConstructor((message, cause) => new 
TableAlreadyExistsException(message, cause)),
+    errorConstructor((message, cause) => new 
TempTableAlreadyExistsException(message, cause)),
+    errorConstructor((message, cause) => new NoSuchDatabaseException(message, 
cause)),
+    errorConstructor((message, cause) => new NoSuchTableException(message, 
cause)))
 
   private def errorInfoToThrowable(info: ErrorInfo, message: String): 
Option[Throwable] = {
     val classes =
@@ -79,7 +86,7 @@ private[client] object GrpcExceptionConverter extends 
JsonUtils {
       .find(errorFactory.contains)
       .map { cls =>
         val constructor = errorFactory.get(cls).get
-        constructor(message, null)
+        constructor(message, None)
       }
   }
 
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 625b6b7c4c5..adbfafd8d7f 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
@@ -115,14 +115,42 @@ class TableAlreadyExistsException private(
   }
 }
 
-class TempTableAlreadyExistsException(errorClass: String, messageParameters: 
Map[String, String],
-                                      cause: Option[Throwable] = None)
-  extends AnalysisException(errorClass, messageParameters, cause = cause) {
+class TempTableAlreadyExistsException 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],
+    cause: Option[Throwable] = None) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters),
+      cause,
+      Some(errorClass),
+      messageParameters)
+  }
+
   def this(table: String) = {
-    this(errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS",
+    this(
+      errorClass = "TEMP_TABLE_OR_VIEW_ALREADY_EXISTS",
       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
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 7da9777be15..d6283a347cc 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
@@ -26,9 +26,39 @@ import org.apache.spark.sql.connector.catalog.Identifier
  * 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.
  */
-case class NoSuchDatabaseException(db: String)
-  extends AnalysisException(errorClass = "SCHEMA_NOT_FOUND",
-    messageParameters = Map("schemaName" -> quoteIdentifier(db)))
+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]) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters),
+      cause = None,
+      Some(errorClass),
+      messageParameters)
+  }
+
+  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])
+  }
+}
 
 // any changes to this class should be backward compatible as it may be used 
by external connectors
 class NoSuchNamespaceException private(


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

Reply via email to