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 4823e031886 [SPARK-44786][SQL][CONNECT] Convert common Spark exceptions
4823e031886 is described below

commit 4823e03188664fd2b34d77f0c5dddfebe8bf6f3d
Author: Yihong He <yihong...@databricks.com>
AuthorDate: Tue Aug 22 19:45:11 2023 +0900

    [SPARK-44786][SQL][CONNECT] Convert common Spark exceptions
    
    - Convert common Spark exceptions
    - Extend common Spark exceptions to support single message parameter 
constructor
    
    - Achieve similar exception conversion coverage as [Python 
Client](https://github.com/apache/spark/blob/master/python/pyspark/errors/exceptions/connect.py#L57-L89)
    
    No
    
    - Existing tests
    
    Closes #42472 from heyihong/SPARK-44786.
    
    Authored-by: Yihong He <yihong...@databricks.com>
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
    (cherry picked from commit dc900b47556dc432f494ad465abdd59fc645734d)
    Signed-off-by: Hyukjin Kwon <gurwls...@apache.org>
---
 .../scala/org/apache/spark/SparkException.scala    | 264 +++++++++++++++++----
 .../spark/sql/ClientDataFrameStatSuite.scala       |   4 +-
 .../org/apache/spark/sql/ClientE2ETestSuite.scala  |  10 +-
 .../connect/client/GrpcExceptionConverter.scala    |  18 +-
 4 files changed, 242 insertions(+), 54 deletions(-)

diff --git a/common/utils/src/main/scala/org/apache/spark/SparkException.scala 
b/common/utils/src/main/scala/org/apache/spark/SparkException.scala
index 4dafaba685e..5c5bf17c942 100644
--- a/common/utils/src/main/scala/org/apache/spark/SparkException.scala
+++ b/common/utils/src/main/scala/org/apache/spark/SparkException.scala
@@ -133,50 +133,107 @@ private[spark] case class ExecutorDeadException(message: 
String)
 /**
  * Exception thrown when Spark returns different result after upgrading to a 
new version.
  */
-private[spark] class SparkUpgradeException(
+private[spark] class SparkUpgradeException private(
+  message: String,
+  cause: Option[Throwable],
+  errorClass: Option[String],
+  messageParameters: Map[String, String])
+  extends RuntimeException(message, cause.orNull) with SparkThrowable {
+
+  def this(
     errorClass: String,
     messageParameters: Map[String, String],
-    cause: Throwable)
-  extends RuntimeException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters), cause)
-  with SparkThrowable {
+    cause: Throwable) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters),
+      Option(cause),
+      Option(errorClass),
+      messageParameters
+    )
+  }
+
+  def this(message: String, cause: Option[Throwable]) = {
+    this(
+      message,
+      cause = cause,
+      errorClass = None,
+      messageParameters = Map.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
 }
 
 /**
  * Arithmetic exception thrown from Spark with an error class.
  */
-private[spark] class SparkArithmeticException(
+private[spark] class SparkArithmeticException private(
+    message: String,
+    errorClass: Option[String],
+    messageParameters: Map[String, String],
+    context: Array[QueryContext])
+  extends ArithmeticException(message) with SparkThrowable {
+
+  def this(
     errorClass: String,
     messageParameters: Map[String, String],
     context: Array[QueryContext],
-    summary: String)
-  extends ArithmeticException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters, summary))
-  with SparkThrowable {
+    summary: String) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters, summary),
+      Option(errorClass),
+      messageParameters,
+      context
+    )
+  }
+
+  def this(message: String) = {
+    this(
+      message,
+      errorClass = None,
+      messageParameters = Map.empty,
+      context = Array.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
   override def getQueryContext: Array[QueryContext] = context
 }
 
 /**
  * Unsupported operation exception thrown from Spark with an error class.
  */
-private[spark] class SparkUnsupportedOperationException(
+private[spark] class SparkUnsupportedOperationException private(
+  message: String,
+  errorClass: Option[String],
+  messageParameters: Map[String, String])
+  extends UnsupportedOperationException(message) with SparkThrowable {
+
+  def this(
     errorClass: String,
-    messageParameters: Map[String, String])
-  extends UnsupportedOperationException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters))
-  with SparkThrowable {
+    messageParameters: Map[String, String]) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters),
+      Option(errorClass),
+      messageParameters
+    )
+  }
+
+  def this(message: String) = {
+    this(
+      message,
+      errorClass = None,
+      messageParameters = Map.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
 }
 
 /**
@@ -214,18 +271,38 @@ private[spark] class SparkConcurrentModificationException(
 /**
  * Datetime exception thrown from Spark with an error class.
  */
-private[spark] class SparkDateTimeException(
+private[spark] class SparkDateTimeException private(
+    message: String,
+    errorClass: Option[String],
+    messageParameters: Map[String, String],
+    context: Array[QueryContext])
+  extends DateTimeException(message) with SparkThrowable {
+
+  def this(
     errorClass: String,
     messageParameters: Map[String, String],
     context: Array[QueryContext],
-    summary: String)
-  extends DateTimeException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters, summary))
-  with SparkThrowable {
+    summary: String) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters, summary),
+      Option(errorClass),
+      messageParameters,
+      context
+    )
+  }
+
+  def this(message: String) = {
+    this(
+      message,
+      errorClass = None,
+      messageParameters = Map.empty,
+      context = Array.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
   override def getQueryContext: Array[QueryContext] = context
 }
 
@@ -247,54 +324,122 @@ private[spark] class SparkFileNotFoundException(
 /**
  * Number format exception thrown from Spark with an error class.
  */
-private[spark] class SparkNumberFormatException(
+private[spark] class SparkNumberFormatException private(
+    message: String,
+    errorClass: Option[String],
+    messageParameters: Map[String, String],
+    context: Array[QueryContext])
+  extends NumberFormatException(message)
+  with SparkThrowable {
+
+  def this(
     errorClass: String,
     messageParameters: Map[String, String],
     context: Array[QueryContext],
-    summary: String)
-  extends NumberFormatException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters, summary))
-  with SparkThrowable {
+    summary: String) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters, summary),
+      Option(errorClass),
+      messageParameters,
+      context
+    )
+  }
+
+  def this(message: String) = {
+    this(
+      message,
+      errorClass = None,
+      messageParameters = Map.empty,
+      context = Array.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
   override def getQueryContext: Array[QueryContext] = context
 }
 
 /**
  * Illegal argument exception thrown from Spark with an error class.
  */
-private[spark] class SparkIllegalArgumentException(
+private[spark] class SparkIllegalArgumentException private(
+    message: String,
+    cause: Option[Throwable],
+    errorClass: Option[String],
+    messageParameters: Map[String, String],
+    context: Array[QueryContext])
+  extends IllegalArgumentException(message, cause.orNull)
+  with SparkThrowable {
+
+  def this(
     errorClass: String,
     messageParameters: Map[String, String],
     context: Array[QueryContext] = Array.empty,
     summary: String = "",
-    cause: Throwable = null)
-  extends IllegalArgumentException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters, summary), 
cause)
-  with SparkThrowable {
+    cause: Throwable = null) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters, summary),
+      Option(cause),
+      Option(errorClass),
+      messageParameters,
+      context
+    )
+  }
+
+  def this(message: String, cause: Option[Throwable]) = {
+    this(
+      message,
+      cause = cause,
+      errorClass = None,
+      messageParameters = Map.empty,
+      context = Array.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
   override def getQueryContext: Array[QueryContext] = context
 }
 
-private[spark] class SparkRuntimeException(
+private[spark] class SparkRuntimeException private(
+    message: String,
+    cause: Option[Throwable],
+    errorClass: Option[String],
+    messageParameters: Map[String, String],
+    context: Array[QueryContext])
+  extends RuntimeException(message, cause.orNull)
+    with SparkThrowable {
+
+  def this(
     errorClass: String,
     messageParameters: Map[String, String],
     cause: Throwable = null,
     context: Array[QueryContext] = Array.empty,
-    summary: String = "")
-  extends RuntimeException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters, summary),
-    cause)
-  with SparkThrowable {
+    summary: String = "") = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters, summary),
+      Option(cause),
+      Option(errorClass),
+      messageParameters,
+      context
+    )
+  }
+
+  def this(message: String, cause: Option[Throwable]) = {
+    this(
+      message,
+      cause = cause,
+      errorClass = None,
+      messageParameters = Map.empty,
+      context = Array.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
   override def getQueryContext: Array[QueryContext] = context
 }
 
@@ -335,18 +480,39 @@ private[spark] class SparkSecurityException(
 /**
  * Array index out of bounds exception thrown from Spark with an error class.
  */
-private[spark] class SparkArrayIndexOutOfBoundsException(
+private[spark] class SparkArrayIndexOutOfBoundsException private(
+  message: String,
+  errorClass: Option[String],
+  messageParameters: Map[String, String],
+  context: Array[QueryContext])
+  extends ArrayIndexOutOfBoundsException(message)
+    with SparkThrowable {
+
+  def this(
     errorClass: String,
     messageParameters: Map[String, String],
     context: Array[QueryContext],
-    summary: String)
-  extends ArrayIndexOutOfBoundsException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters, summary))
-  with SparkThrowable {
+    summary: String) = {
+    this(
+      SparkThrowableHelper.getMessage(errorClass, messageParameters, summary),
+      Option(errorClass),
+      messageParameters,
+      context
+    )
+  }
+
+  def this(message: String) = {
+    this(
+      message,
+      errorClass = None,
+      messageParameters = Map.empty,
+      context = Array.empty
+    )
+  }
 
   override def getMessageParameters: java.util.Map[String, String] = 
messageParameters.asJava
 
-  override def getErrorClass: String = errorClass
+  override def getErrorClass: String = errorClass.orNull
   override def getQueryContext: Array[QueryContext] = context
 }
 
diff --git 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala
 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala
index 7035dc99148..2f4e1aa9bd0 100644
--- 
a/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala
+++ 
b/connector/connect/client/jvm/src/test/scala/org/apache/spark/sql/ClientDataFrameStatSuite.scala
@@ -21,7 +21,7 @@ import java.util.Random
 
 import org.scalatest.matchers.must.Matchers._
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkException, SparkIllegalArgumentException}
 import org.apache.spark.sql.connect.client.util.RemoteSparkSession
 
 class ClientDataFrameStatSuite extends RemoteSparkSession {
@@ -87,7 +87,7 @@ class ClientDataFrameStatSuite extends RemoteSparkSession {
 
     val results = df.stat.cov("singles", "doubles")
     assert(math.abs(results - 55.0 / 3) < 1e-12)
-    intercept[SparkException] {
+    intercept[SparkIllegalArgumentException] {
       df.stat.cov("singles", "letters") // doesn't accept non-numerical 
dataTypes
     }
     val decimalData = Seq.tabulate(6)(i => (BigDecimal(i % 3), BigDecimal(i % 
2))).toDF("a", "b")
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 1bfc1cfc7a8..86973b82e72 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
@@ -29,8 +29,8 @@ import org.apache.commons.lang3.{JavaVersion, SystemUtils}
 import org.scalactic.TolerantNumerics
 import org.scalatest.PrivateMethodTester
 
+import org.apache.spark.{SparkArithmeticException, SparkException}
 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
@@ -44,6 +44,14 @@ import org.apache.spark.sql.types._
 
 class ClientE2ETestSuite extends RemoteSparkSession with SQLHelper with 
PrivateMethodTester {
 
+  test("throw SparkArithmeticException") {
+    withSQLConf("spark.sql.ansi.enabled" -> "true") {
+      intercept[SparkArithmeticException] {
+        spark.sql("select 1/0").collect()
+      }
+    }
+  }
+
   test("throw NoSuchDatabaseException") {
     intercept[NoSuchDatabaseException] {
       spark.sql("use database123")
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 cf92a53847e..672d31be954 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
@@ -16,6 +16,8 @@
  */
 package org.apache.spark.sql.connect.client
 
+import java.time.DateTimeException
+
 import scala.collection.JavaConverters._
 import scala.reflect.ClassTag
 
@@ -23,7 +25,7 @@ import com.google.rpc.ErrorInfo
 import io.grpc.StatusRuntimeException
 import io.grpc.protobuf.StatusProto
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkArithmeticException, 
SparkArrayIndexOutOfBoundsException, SparkDateTimeException, SparkException, 
SparkIllegalArgumentException, SparkNumberFormatException, 
SparkRuntimeException, SparkUnsupportedOperationException, 
SparkUpgradeException}
 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
@@ -76,7 +78,19 @@ private[client] object GrpcExceptionConverter extends 
JsonUtils {
     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)))
+    errorConstructor((message, cause) => new NoSuchTableException(message, 
cause)),
+    errorConstructor[NumberFormatException]((message, _) =>
+      new SparkNumberFormatException(message)),
+    errorConstructor[IllegalArgumentException]((message, cause) =>
+      new SparkIllegalArgumentException(message, cause)),
+    errorConstructor[ArithmeticException]((message, _) => new 
SparkArithmeticException(message)),
+    errorConstructor[UnsupportedOperationException]((message, _) =>
+      new SparkUnsupportedOperationException(message)),
+    errorConstructor[ArrayIndexOutOfBoundsException]((message, _) =>
+      new SparkArrayIndexOutOfBoundsException(message)),
+    errorConstructor[DateTimeException]((message, _) => new 
SparkDateTimeException(message)),
+    errorConstructor((message, cause) => new SparkRuntimeException(message, 
cause)),
+    errorConstructor((message, cause) => new SparkUpgradeException(message, 
cause)))
 
   private def errorInfoToThrowable(info: ErrorInfo, message: String): 
Option[Throwable] = {
     val classes =


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

Reply via email to