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 f1eecd357861 [SPARK-50485][SQL] Unwrap SparkThrowable in 
(Unchecked)ExecutionException thrown by tableRelationCache
f1eecd357861 is described below

commit f1eecd3578612da510748da8682c199aa6109eac
Author: Kent Yao <[email protected]>
AuthorDate: Wed Dec 4 18:53:35 2024 +0100

    [SPARK-50485][SQL] Unwrap SparkThrowable in (Unchecked)ExecutionException 
thrown by tableRelationCache
    
    ### What changes were proposed in this pull request?
    
    This PR unwraps SparkThrowable in (Unchecked)ExecutionException thrown by 
tableRelationCache
    ### Why are the changes needed?
    
    The guava cache library wraps exceptions thrown by `c: 
Callable[LogicalPlan]` as `(Unchecked)ExecutionException`s. This makes our code 
paths of special handling for SparkThrowable inoperative. For example, this 
kind of error in spark-sql cli is very lengthy.
    
    ### Does this PR introduce _any_ user-facing change?
    
    Yes, when you visit an invalid plan in the table cache layer, you will get 
the original spark error instead of getting them under 
`(Unchecked)ExecutionException`s
    
    ### How was this patch tested?
    new tests
    
    ### Was this patch authored or co-authored using generative AI tooling?
    
    no
    
    Closes #49050 from yaooqinn/SPARK-50485.
    
    Authored-by: Kent Yao <[email protected]>
    Signed-off-by: Max Gekk <[email protected]>
---
 .../spark/sql/catalyst/catalog/SessionCatalog.scala      | 14 ++++++++++----
 .../spark/sql/errors/QueryExecutionErrorsSuite.scala     | 16 ++++++++++++++++
 2 files changed, 26 insertions(+), 4 deletions(-)

diff --git 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index a0f7af10fefa..d87678ac3411 100644
--- 
a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ 
b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -19,18 +19,18 @@ package org.apache.spark.sql.catalyst.catalog
 
 import java.net.URI
 import java.util.Locale
-import java.util.concurrent.Callable
-import java.util.concurrent.TimeUnit
+import java.util.concurrent.{Callable, ExecutionException, TimeUnit}
 import javax.annotation.concurrent.GuardedBy
 
 import scala.collection.mutable
 import scala.util.{Failure, Success, Try}
 
 import com.google.common.cache.{Cache, CacheBuilder}
+import com.google.common.util.concurrent.UncheckedExecutionException
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
 
-import org.apache.spark.SparkException
+import org.apache.spark.{SparkException, SparkThrowable}
 import org.apache.spark.internal.Logging
 import org.apache.spark.sql.catalyst._
 import org.apache.spark.sql.catalyst.analysis._
@@ -210,7 +210,13 @@ class SessionCatalog(
 
   /** This method provides a way to get a cached plan. */
   def getCachedPlan(t: QualifiedTableName, c: Callable[LogicalPlan]): 
LogicalPlan = {
-    tableRelationCache.get(t, c)
+    try {
+      tableRelationCache.get(t, c)
+    } catch {
+      case e @ (_: ExecutionException | _: UncheckedExecutionException)
+          if e.getCause != null && e.getCause.isInstanceOf[SparkThrowable] =>
+        throw e.getCause
+    }
   }
 
   /** This method provides a way to get a cached plan if the key exists. */
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 1adb1fdf0503..9abda729c02e 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
@@ -1258,6 +1258,22 @@ class QueryExecutionErrorsSuite
       )
     )
   }
+
+  test("SPARK-50485: Unwrap SparkThrowable in UEE thrown by 
tableRelationCache") {
+    withTable("t") {
+      sql("CREATE TABLE t (a INT)")
+      checkError(
+        exception = intercept[SparkUnsupportedOperationException] {
+          sql("ALTER TABLE t SET LOCATION 'https://mister/spark'")
+        },
+        condition = "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM",
+        parameters = Map(
+          "path" -> "https://mister/spark";,
+          "fileSystemClass" -> "org.apache.hadoop.fs.http.HttpsFileSystem",
+          "method" -> "listStatus"))
+      sql("ALTER TABLE t SET LOCATION '/mister/spark'")
+    }
+  }
 }
 
 class FakeFileSystemSetPermission extends LocalFileSystem {


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to