This is an automated email from the ASF dual-hosted git repository.
dongjoon 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 33f248ce2364 [SPARK-50458][CORE][SQL] Proper error handling for
unsupported file system when reading files
33f248ce2364 is described below
commit 33f248ce2364b3cacfb830561bd299270358fcc7
Author: Kent Yao <[email protected]>
AuthorDate: Mon Dec 2 15:46:35 2024 -0800
[SPARK-50458][CORE][SQL] Proper error handling for unsupported file system
when reading files
### What changes were proposed in this pull request?
Add FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM for reading a valid file but
on an unsupported file system.
### Why are the changes needed?
For these errors, we report ambiguous errors now. For example, querying
json.`https://a/b/c.json`, will report UNSUPPORTED_DATASOURCE_FOR_DIRECT_QUERY
but json is actually supported.
### Does this PR introduce _any_ user-facing change?
Yes, a meaningful error is given when visiting an unsupported file system.
### How was this patch tested?
new tests
### Was this patch authored or co-authored using generative AI tooling?
no
Closes #49016 from yaooqinn/SPARK-50458.
Authored-by: Kent Yao <[email protected]>
Signed-off-by: Dongjoon Hyun <[email protected]>
---
.../src/main/resources/error/error-conditions.json | 5 +++++
.../org/apache/spark/util/HadoopFSUtils.scala | 7 +++++++
.../spark/sql/execution/datasources/rules.scala | 5 +++++
.../analyzer-results/sql-on-files.sql.out | 22 ++++++++++++++++++++
.../resources/sql-tests/inputs/sql-on-files.sql | 2 ++
.../sql-tests/results/sql-on-files.sql.out | 24 ++++++++++++++++++++++
.../execution/datasources/DataSourceSuite.scala | 13 ++++++++++++
7 files changed, 78 insertions(+)
diff --git a/common/utils/src/main/resources/error/error-conditions.json
b/common/utils/src/main/resources/error/error-conditions.json
index 632d43b4d105..157989c09d09 100644
--- a/common/utils/src/main/resources/error/error-conditions.json
+++ b/common/utils/src/main/resources/error/error-conditions.json
@@ -1476,6 +1476,11 @@
"message" : [
"Data type mismatches when reading Parquet column <column>. Expected
Spark type <expectedType>, actual Parquet type <actualType>."
]
+ },
+ "UNSUPPORTED_FILE_SYSTEM" : {
+ "message" : [
+ "The file system <fileSystemClass> hasn't implemented <method>."
+ ]
}
},
"sqlState" : "KD001"
diff --git a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
index f8f5bb4f72a4..5e50361b278a 100644
--- a/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
+++ b/core/src/main/scala/org/apache/spark/util/HadoopFSUtils.scala
@@ -241,6 +241,13 @@ private[spark] object HadoopFSUtils extends Logging {
logWarning(log"The directory ${MDC(PATH, path)} " +
log"was not found. Was it deleted very recently?")
Array.empty[FileStatus]
+ case u: UnsupportedOperationException =>
+ throw new SparkUnsupportedOperationException(
+ errorClass = "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM",
+ messageParameters = Map(
+ "path" -> path.toString,
+ "fileSystemClass" -> fs.getClass.getName,
+ "method" -> u.getStackTrace.head.getMethodName))
}
val filteredStatuses =
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
index fcc3a257cd2d..713161cc49ce 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala
@@ -22,6 +22,7 @@ import java.util.Locale
import scala.collection.mutable.{HashMap, HashSet}
import scala.jdk.CollectionConverters._
+import org.apache.spark.SparkUnsupportedOperationException
import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog._
@@ -54,6 +55,10 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends
Rule[LogicalPlan] {
val ds = resolveDataSource(u)
Some(LogicalRelation(ds.resolveRelation()))
} catch {
+ case e: SparkUnsupportedOperationException =>
+ u.failAnalysis(
+ errorClass = e.getCondition,
+ messageParameters = e.getMessageParameters.asScala.toMap)
case _: ClassNotFoundException => None
case e: Exception if !e.isInstanceOf[AnalysisException] =>
// the provider is valid, but failed to create a logical plan
diff --git
a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out
b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out
index b098a9758fe4..329fbd49b92d 100644
---
a/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out
+++
b/sql/core/src/test/resources/sql-tests/analyzer-results/sql-on-files.sql.out
@@ -231,3 +231,25 @@ DROP DATABASE sql_on_files
-- !query analysis
DropNamespace false, false
+- ResolvedNamespace V2SessionCatalog(spark_catalog), [sql_on_files]
+
+
+-- !query
+SELECT * FROM
json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`
+-- !query analysis
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM",
+ "sqlState" : "KD001",
+ "messageParameters" : {
+ "fileSystemClass" : "org.apache.hadoop.fs.http.HttpsFileSystem",
+ "method" : "listStatus",
+ "path" :
"https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 128,
+ "fragment" :
"json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`"
+ } ]
+}
diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql
b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql
index 8a00e4400e6b..c3a16ca577ee 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/sql-on-files.sql
@@ -28,3 +28,5 @@ SELECT * FROM
json.`${spark.sql.warehouse.dir}/sql_on_files.db/test_json`;
DROP TABLE sql_on_files.test_json;
DROP DATABASE sql_on_files;
+
+SELECT * FROM
json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`;
diff --git a/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out
b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out
index fc8f44bc22fe..71d4216ea207 100644
--- a/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/sql-on-files.sql.out
@@ -257,3 +257,27 @@ DROP DATABASE sql_on_files
struct<>
-- !query output
+
+
+-- !query
+SELECT * FROM
json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`
+-- !query schema
+struct<>
+-- !query output
+org.apache.spark.sql.AnalysisException
+{
+ "errorClass" : "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM",
+ "sqlState" : "KD001",
+ "messageParameters" : {
+ "fileSystemClass" : "org.apache.hadoop.fs.http.HttpsFileSystem",
+ "method" : "listStatus",
+ "path" :
"https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json"
+ },
+ "queryContext" : [ {
+ "objectType" : "",
+ "objectName" : "",
+ "startIndex" : 15,
+ "stopIndex" : 128,
+ "fragment" :
"json.`https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/src/main/resources/employees.json`"
+ } ]
+}
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala
index fd9d31e7a594..d2acdcfc6205 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/DataSourceSuite.scala
@@ -25,6 +25,7 @@ import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileStatus, Path, RawLocalFileSystem}
import org.scalatest.PrivateMethodTester
+import org.apache.spark.SparkUnsupportedOperationException
import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.util.Utils
@@ -207,6 +208,18 @@ class DataSourceSuite extends SharedSparkSession with
PrivateMethodTester {
Utils.deleteRecursively(baseDir)
}
}
+
+ test("SPARK-50458: Proper error handling for unsupported file system") {
+ val loc =
"https://raw.githubusercontent.com/apache/spark/refs/heads/master/examples/" +
+ "src/main/resources/employees.json"
+ checkError(exception = intercept[SparkUnsupportedOperationException](
+ sql(s"CREATE TABLE HTTP USING JSON LOCATION '$loc'")),
+ condition = "FAILED_READ_FILE.UNSUPPORTED_FILE_SYSTEM",
+ parameters = Map(
+ "path" -> loc,
+ "fileSystemClass" -> "org.apache.hadoop.fs.http.HttpsFileSystem",
+ "method" -> "listStatus"))
+ }
}
object TestPaths {
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]