Skip to content

Commit f1eecd3

Browse files
yaooqinnMaxGekk
authored andcommitted
[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 apache#49050 from yaooqinn/SPARK-50485. Authored-by: Kent Yao <[email protected]> Signed-off-by: Max Gekk <[email protected]>
1 parent 812a9ad commit f1eecd3

File tree

2 files changed

+26
-4
lines changed

2 files changed

+26
-4
lines changed

sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala

Lines changed: 10 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -19,18 +19,18 @@ package org.apache.spark.sql.catalyst.catalog
1919

2020
import java.net.URI
2121
import java.util.Locale
22-
import java.util.concurrent.Callable
23-
import java.util.concurrent.TimeUnit
22+
import java.util.concurrent.{Callable, ExecutionException, TimeUnit}
2423
import javax.annotation.concurrent.GuardedBy
2524

2625
import scala.collection.mutable
2726
import scala.util.{Failure, Success, Try}
2827

2928
import com.google.common.cache.{Cache, CacheBuilder}
29+
import com.google.common.util.concurrent.UncheckedExecutionException
3030
import org.apache.hadoop.conf.Configuration
3131
import org.apache.hadoop.fs.Path
3232

33-
import org.apache.spark.SparkException
33+
import org.apache.spark.{SparkException, SparkThrowable}
3434
import org.apache.spark.internal.Logging
3535
import org.apache.spark.sql.catalyst._
3636
import org.apache.spark.sql.catalyst.analysis._
@@ -210,7 +210,13 @@ class SessionCatalog(
210210

211211
/** This method provides a way to get a cached plan. */
212212
def getCachedPlan(t: QualifiedTableName, c: Callable[LogicalPlan]): LogicalPlan = {
213-
tableRelationCache.get(t, c)
213+
try {
214+
tableRelationCache.get(t, c)
215+
} catch {
216+
case e @ (_: ExecutionException | _: UncheckedExecutionException)
217+
if e.getCause != null && e.getCause.isInstanceOf[SparkThrowable] =>
218+
throw e.getCause
219+
}
214220
}
215221

216222
/** This method provides a way to get a cached plan if the key exists. */

sql/core/src/test/scala/org/apache/spark/sql/errors/QueryExecutionErrorsSuite.scala

Lines changed: 16 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1258,6 +1258,22 @@ class QueryExecutionErrorsSuite
12581258
)
12591259
)
12601260
}
1261+
1262+
test("SPARK-50485: Unwrap SparkThrowable in UEE thrown by tableRelationCache") {
1263+
withTable("t") {
1264+
sql("CREATE TABLE t (a INT)")
1265+
checkError(
1266+
exception = intercept[SparkUnsupportedOperationException] {
1267+
sql("ALTER TABLE t SET LOCATION 'https://mister/spark'")
1268+
},
1269+
condition = "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM",
1270+
parameters = Map(
1271+
"path" -> "https://mister/spark",
1272+
"fileSystemClass" -> "org.apache.hadoop.fs.http.HttpsFileSystem",
1273+
"method" -> "listStatus"))
1274+
sql("ALTER TABLE t SET LOCATION '/mister/spark'")
1275+
}
1276+
}
12611277
}
12621278

12631279
class FakeFileSystemSetPermission extends LocalFileSystem {

0 commit comments

Comments
 (0)