This is an automated email from the ASF dual-hosted git repository.
wenchen pushed a commit to branch branch-3.4
in repository https://gitbox.apache.org/repos/asf/spark.git
The following commit(s) were added to refs/heads/branch-3.4 by this push:
new b35e42f9a79 [SPARK-43284][SQL][FOLLOWUP] Return URI encoded path, and
add a test
b35e42f9a79 is described below
commit b35e42f9a79c2c1f2b3edce64ae9b97e272a444a
Author: David Lewis <[email protected]>
AuthorDate: Fri May 5 16:50:07 2023 +0800
[SPARK-43284][SQL][FOLLOWUP] Return URI encoded path, and add a test
Return URI encoded path, and add a test
Fix regression in spark 3.4.
Yes, fixes a regression in `_metadata.file_path`.
New explicit test.
Closes #41054 from databricks-david-lewis/SPARK-43284-2.
Lead-authored-by: David Lewis <[email protected]>
Co-authored-by: Wenchen Fan <[email protected]>
Signed-off-by: Wenchen Fan <[email protected]>
(cherry picked from commit 79d5d908e5dd7d6ab6755c46d4de3ed2fcdf9e6b)
Signed-off-by: Wenchen Fan <[email protected]>
---
.../spark/sql/execution/datasources/FileFormat.scala | 4 ++--
.../sql/execution/datasources/FileScanRDD.scala | 4 ++--
.../datasources/FileMetadataStructSuite.scala | 20 +++++++++++++++++++-
3 files changed, 23 insertions(+), 5 deletions(-)
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
index decd0c30b00..3a5cb1e8f47 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala
@@ -256,8 +256,8 @@ object FileFormat {
fieldNames.zipWithIndex.foreach { case (name, i) =>
name match {
case FILE_PATH =>
- // Use new Path(Path.toString).toString as a form of canonicalization
- val pathString = new Path(filePath.toString).toString
+ // Use `new Path(Path.toString)` as a form of canonicalization
+ val pathString = new Path(filePath.toString).toUri.toString
row.update(i, UTF8String.fromString(pathString))
case FILE_NAME =>
val fileName =
filePath.toUri.getRawPath.split("/").lastOption.getOrElse("")
diff --git
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
index eebd92dad6e..25bfadab6e0 100644
---
a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
+++
b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala
@@ -159,8 +159,8 @@ class FileScanRDD(
metadataColumns.map(_.name).map {
case FILE_PATH =>
val columnVector = new ConstantColumnVector(c.numRows(),
StringType)
- // Use new Path(Path.toString).toString as a form of
canonicalization
- val pathString = new Path(path.toString).toString
+ // Use `new Path(Path.toString)` as a form of canonicalization
+ val pathString = new Path(path.toString).toUri.toString
columnVector.setUtf8String(UTF8String.fromString(pathString))
columnVector
case FILE_NAME =>
diff --git
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala
index ee207f68e34..5fccf393e23 100644
---
a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala
+++
b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileMetadataStructSuite.scala
@@ -881,7 +881,25 @@ class FileMetadataStructSuite extends QueryTest with
SharedSparkSession {
Seq("parquet", "json", "csv", "text", "orc").foreach { format =>
- test(s"metadata file path and name are url encoded for format: $format") {
+ test(s"metadata file path is url encoded for format: $format") {
+ withTempPath { f =>
+ val dirWithSpace = s"$f/with space"
+ spark.range(10)
+ .selectExpr("cast(id as string) as str")
+ .repartition(1)
+ .write
+ .format(format)
+ .mode("append")
+ .save(dirWithSpace)
+
+ val encodedPath = SparkPath.fromPathString(dirWithSpace).urlEncoded
+ val df = spark.read.format(format).load(dirWithSpace)
+ val metadataPath = df.select(METADATA_FILE_PATH).as[String].head()
+ assert(metadataPath.contains(encodedPath))
+ }
+ }
+
+ test(s"metadata file name is url encoded for format: $format") {
val suffix = if (format == "text") ".txt" else s".$format"
withTempPath { f =>
val dirWithSpace = s"$f/with space"
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]