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

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


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 1853eb117e2 [SPARK-39187][SQL][3.3] Remove `SparkIllegalStateException`
1853eb117e2 is described below

commit 1853eb117e24bcc0509d275c4caca6c033bf0ab9
Author: Max Gekk <max.g...@gmail.com>
AuthorDate: Mon May 16 11:39:37 2022 +0300

    [SPARK-39187][SQL][3.3] Remove `SparkIllegalStateException`
    
    ### What changes were proposed in this pull request?
    Remove `SparkIllegalStateException` and replace it by 
`IllegalStateException` where it was used.
    
    This is a backport of https://github.com/apache/spark/pull/36550.
    
    ### Why are the changes needed?
    To improve code maintenance and be consistent to other places where 
`IllegalStateException` is used in illegal states (for instance, see 
https://github.com/apache/spark/pull/36524). After the PR 
https://github.com/apache/spark/pull/36500, the exception is substituted by 
`SparkException` w/ the `INTERNAL_ERROR` error class.
    
    ### Does this PR introduce _any_ user-facing change?
    No. Users shouldn't face to the exception in regular cases.
    
    ### How was this patch tested?
    By running the affected test suites:
    ```
    $ build/sbt "sql/test:testOnly *QueryExecutionErrorsSuite*"
    $ build/sbt "test:testOnly *ArrowUtilsSuite"
    ```
    
    Authored-by: Max Gekk <max.gekkgmail.com>
    Signed-off-by: Max Gekk <max.gekkgmail.com>
    (cherry picked from commit 1a90512f605c490255f7b38215c207e64621475b)
    Signed-off-by: Max Gekk <max.gekkgmail.com>
    
    Closes #36558 from MaxGekk/remove-SparkIllegalStateException-3.3.
    
    Authored-by: Max Gekk <max.g...@gmail.com>
    Signed-off-by: Max Gekk <max.g...@gmail.com>
---
 core/src/main/scala/org/apache/spark/SparkException.scala  | 12 ------------
 .../apache/spark/sql/catalyst/analysis/CheckAnalysis.scala |  6 +++---
 .../org/apache/spark/sql/errors/QueryExecutionErrors.scala | 11 +++--------
 .../main/scala/org/apache/spark/sql/util/ArrowUtils.scala  |  9 +++------
 .../scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala  |  4 ++--
 .../spark/sql/errors/QueryExecutionErrorsSuite.scala       | 14 --------------
 6 files changed, 11 insertions(+), 45 deletions(-)

diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala 
b/core/src/main/scala/org/apache/spark/SparkException.scala
index 8442c8eb8d3..ed6e811a4cc 100644
--- a/core/src/main/scala/org/apache/spark/SparkException.scala
+++ b/core/src/main/scala/org/apache/spark/SparkException.scala
@@ -158,18 +158,6 @@ private[spark] class SparkFileAlreadyExistsException(
   override def getErrorClass: String = errorClass
 }
 
-/**
- * Illegal state exception thrown from Spark with an error class.
- */
-private[spark] class SparkIllegalStateException(
-    errorClass: String,
-    messageParameters: Array[String])
-  extends IllegalStateException(
-    SparkThrowableHelper.getMessage(errorClass, messageParameters)) with 
SparkThrowable {
-
-  override def getErrorClass: String = errorClass
-}
-
 /**
  * File not found exception thrown from Spark with an error class.
  */
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
index ff40272682e..f89fbe59af6 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala
@@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.trees.TreeNodeTag
 import org.apache.spark.sql.catalyst.util.{CharVarcharUtils, StringUtils, 
TypeUtils}
 import org.apache.spark.sql.connector.catalog.{LookupCatalog, 
SupportsPartitionManagement}
-import org.apache.spark.sql.errors.{QueryCompilationErrors, 
QueryExecutionErrors}
+import org.apache.spark.sql.errors.QueryCompilationErrors
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.types._
 import org.apache.spark.sql.util.SchemaUtils
@@ -571,8 +571,8 @@ trait CheckAnalysis extends PredicateHelper with 
LookupCatalog {
                  |in operator 
${operator.simpleString(SQLConf.get.maxToStringFields)}
                """.stripMargin)
 
-          case _: UnresolvedHint =>
-            throw 
QueryExecutionErrors.logicalHintOperatorNotRemovedDuringAnalysisError
+          case _: UnresolvedHint => throw new IllegalStateException(
+            "Logical hint operator should be removed during analysis.")
 
           case f @ Filter(condition, _)
             if PlanHelper.specialExpressionsInUnsupportedOperator(f).nonEmpty 
=>
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
index df5959283eb..cf87094ad27 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/errors/QueryExecutionErrors.scala
@@ -34,7 +34,7 @@ import org.apache.hadoop.fs.permission.FsPermission
 import org.codehaus.commons.compiler.CompileException
 import org.codehaus.janino.InternalCompilerException
 
-import org.apache.spark.{Partition, SparkArithmeticException, 
SparkArrayIndexOutOfBoundsException, SparkClassNotFoundException, 
SparkConcurrentModificationException, SparkDateTimeException, SparkException, 
SparkFileAlreadyExistsException, SparkFileNotFoundException, 
SparkIllegalArgumentException, SparkIllegalStateException, 
SparkIndexOutOfBoundsException, SparkNoSuchElementException, 
SparkNoSuchMethodException, SparkNumberFormatException, SparkRuntimeException, 
SparkSecurityException, Sp [...]
+import org.apache.spark.{Partition, SparkArithmeticException, 
SparkArrayIndexOutOfBoundsException, SparkClassNotFoundException, 
SparkConcurrentModificationException, SparkDateTimeException, SparkException, 
SparkFileAlreadyExistsException, SparkFileNotFoundException, 
SparkIllegalArgumentException, SparkIndexOutOfBoundsException, 
SparkNoSuchElementException, SparkNoSuchMethodException, 
SparkNumberFormatException, SparkRuntimeException, SparkSecurityException, 
SparkSQLException, SparkSQLFea [...]
 import org.apache.spark.executor.CommitDeniedException
 import org.apache.spark.launcher.SparkLauncher
 import org.apache.spark.memory.SparkOutOfMemoryError
@@ -68,12 +68,6 @@ import org.apache.spark.util.CircularBuffer
  */
 object QueryExecutionErrors extends QueryErrorsBase {
 
-  def logicalHintOperatorNotRemovedDuringAnalysisError(): Throwable = {
-    new SparkIllegalStateException(errorClass = "INTERNAL_ERROR",
-      messageParameters = Array(
-        "Internal error: logical hint operator should have been removed during 
analysis"))
-  }
-
   def cannotEvaluateExpressionError(expression: Expression): Throwable = {
     new SparkUnsupportedOperationException(errorClass = "INTERNAL_ERROR",
       messageParameters = Array(s"Cannot evaluate expression: $expression"))
@@ -137,7 +131,8 @@ object QueryExecutionErrors extends QueryErrorsBase {
   }
 
   def cannotParseDecimalError(): Throwable = {
-    new SparkIllegalStateException(errorClass = "CANNOT_PARSE_DECIMAL",
+    new SparkRuntimeException(
+      errorClass = "CANNOT_PARSE_DECIMAL",
       messageParameters = Array.empty)
   }
 
diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
index 4254c045ca6..b8f77c3646c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala
@@ -47,12 +47,9 @@ private[sql] object ArrowUtils {
     case BinaryType => ArrowType.Binary.INSTANCE
     case DecimalType.Fixed(precision, scale) => new 
ArrowType.Decimal(precision, scale)
     case DateType => new ArrowType.Date(DateUnit.DAY)
-    case TimestampType =>
-      if (timeZoneId == null) {
-        throw 
QueryExecutionErrors.timeZoneIdNotSpecifiedForTimestampTypeError()
-      } else {
-        new ArrowType.Timestamp(TimeUnit.MICROSECOND, timeZoneId)
-      }
+    case TimestampType if timeZoneId == null =>
+      throw new IllegalStateException("Missing timezoneId where it is 
mandatory.")
+    case TimestampType => new ArrowType.Timestamp(TimeUnit.MICROSECOND, 
timeZoneId)
     case TimestampNTZType =>
       new ArrowType.Timestamp(TimeUnit.MICROSECOND, null)
     case NullType => ArrowType.Null.INSTANCE
diff --git 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
index 642b387b88e..6dd02afe19b 100644
--- 
a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
+++ 
b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala
@@ -50,10 +50,10 @@ class ArrowUtilsSuite extends SparkFunSuite {
     roundtrip(DateType)
     roundtrip(YearMonthIntervalType())
     roundtrip(DayTimeIntervalType())
-    val tsExMsg = intercept[UnsupportedOperationException] {
+    val tsExMsg = intercept[IllegalStateException] {
       roundtrip(TimestampType)
     }
-    assert(tsExMsg.getMessage.contains("timeZoneId"))
+    assert(tsExMsg.getMessage.contains("timezoneId"))
   }
 
   test("timestamp") {
diff --git 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
 
b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
index af6402e4fe2..96a29f6dab6 100644
--- 
a/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
+++ 
b/sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala
@@ -25,8 +25,6 @@ import org.apache.spark.sql.functions.{lit, lower, struct, 
sum}
 import org.apache.spark.sql.internal.SQLConf
 import org.apache.spark.sql.internal.SQLConf.LegacyBehaviorPolicy.EXCEPTION
 import org.apache.spark.sql.test.SharedSparkSession
-import org.apache.spark.sql.types.{StructType, TimestampType}
-import org.apache.spark.sql.util.ArrowUtils
 
 class QueryExecutionErrorsSuite extends QueryTest
   with ParquetTest with OrcTest with SharedSparkSession {
@@ -228,18 +226,6 @@ class QueryExecutionErrorsSuite extends QueryTest
     }
   }
 
-  test("UNSUPPORTED_OPERATION: timeZoneId not specified while converting 
TimestampType to Arrow") {
-    val schema = new StructType().add("value", TimestampType)
-    val e = intercept[SparkUnsupportedOperationException] {
-      ArrowUtils.toArrowSchema(schema, null)
-    }
-
-    assert(e.getErrorClass === "UNSUPPORTED_OPERATION")
-    assert(e.getMessage === "The operation is not supported: " +
-      "\"TIMESTAMP\" must supply timeZoneId parameter " +
-      "while converting to the arrow timestamp type.")
-  }
-
   test("UNSUPPORTED_OPERATION - SPARK-36346: can't read Timestamp as 
TimestampNTZ") {
     withTempPath { file =>
       sql("select timestamp_ltz'2019-03-21 
00:02:03'").write.orc(file.getCanonicalPath)


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

Reply via email to