Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[SPARK-39910][SQL] Delegate path qualification to filesystem during DataSource file path globbing #43463

Conversation

tigrulya-exe
Copy link
Contributor

@tigrulya-exe tigrulya-exe commented Oct 20, 2023

What changes were proposed in this pull request?

In current version DataSource#checkAndGlobPathIfNecessary qualifies paths via Path#makeQualified and PartitioningAwareFileIndex qualifies via FileSystem#makeQualified. Most FileSystem implementations simply delegate to Path#makeQualified, but others, like HarFileSystem contain fs-specific logic, that can produce different result. Such inconsistencies can lead to a situation, when spark can't find partitions of the source file, because qualified paths, built by Path and FileSystem are different. Therefore, for uniformity, the FileSystem path qualification should be used in DataSource#checkAndGlobPathIfNecessary.

Why are the changes needed?

Allow users to read files from hadoop archives (.har) using DataFrameReader API

Does this PR introduce any user-facing change?

No

How was this patch tested?

New tests were added in DataSourceSuite and DataFrameReaderWriterSuite

Was this patch authored or co-authored using generative AI tooling?

No

@github-actions github-actions bot added the SQL label Oct 20, 2023
val jsonRelativeHarPath = new Path("/test.json")
val parquetRelativeHarPath = new Path("/test.parquet")
val orcRelativeHarPath = new Path("/test.orc")
val globeRelativeHarPath = new Path("/test.*")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Do we really need to test all the file format?

Copy link
Contributor Author

@tigrulya-exe tigrulya-exe Oct 20, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

They're needed to test glob path. I decided to reuse har from DataFrameReaderWriterSuite tests instead of creating new archive.

@beliefer
Copy link
Contributor

@tigrulya-exe Please re-trigger GA tests.

@tigrulya-exe
Copy link
Contributor Author

@beliefer I re-ran tests several times, but they failed either due to lack of resources or due to flaky ProtobufCatalystDataConversionSuite. I will rebase on #43493 after it will be merged

@beliefer
Copy link
Contributor

Please fix the conflicts.

@tigrulya-exe tigrulya-exe force-pushed the SPARK-39910-use-fs-path-qualification branch from 898e77e to 4532b46 Compare October 25, 2023 08:41
@tigrulya-exe
Copy link
Contributor Author

@beliefer Hi! I fixed the conflicts and rebased on master

Copy link
Contributor

@beliefer beliefer left a comment

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

@tigrulya-exe
Copy link
Contributor Author

@cloud-fan Hi! Could you take a look please?

@tigrulya-exe tigrulya-exe force-pushed the SPARK-39910-use-fs-path-qualification branch from 4532b46 to a005f65 Compare January 18, 2024 08:40
@github-actions github-actions bot added the BUILD label Jan 18, 2024
@tigrulya-exe
Copy link
Contributor Author

@cloud-fan Hi! I've rebased on master and fixed conflicts. Could you please take a look?

@cloud-fan
Copy link
Contributor

The fix is straightforward but the test is convoluted. How do you test HarFileSystem? I can't find any code setting the file system, but only the directory name contains har.

@tigrulya-exe
Copy link
Contributor Author

tigrulya-exe commented Feb 6, 2024

@cloud-fan we construct absolute file paths with har:// scheme in the DataSourceSuite#buildFullHarPaths method and then check that they're correctly qualified. Then we test reading files inside har archive by their absolute paths.

We don't need to create or test the HarFileSystem itself, it is extracted from the path in the DataSource#checkAndGlobPathIfNecessary method.

test("SPARK-39910: read files from Hadoop archives") {
val fileSchema = new StructType().add("str", StringType)
val harPath = testFile("test-data/test-archive.har")
.replaceFirst("file:/", "har:/")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

So Spark works with har:/ paths out of the box? BTW, I think this test is good enough, we don't need to add more tests in DataSourceSuite.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, the HarFileSystem support is included in the HDFS client by default. Ok, removed tests from DataSourceSuite, left only MockFileSystem#getUri method to correctly qualify paths with mockFs:// scheme.

val harPath = testFile("test-data/test-archive.har")
.replaceFirst("file:/", "har:/")

testRead(spark.read.textFile(s"$harPath/test.txt").toDF(), data, textSchema)
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Since we only want to test path globbing, I think testing one file format is sufficient.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ok, removed file formats other than csv

@tigrulya-exe tigrulya-exe force-pushed the SPARK-39910-use-fs-path-qualification branch from a005f65 to 8161581 Compare February 7, 2024 14:33
@@ -214,4 +216,6 @@ class MockFileSystem extends RawLocalFileSystem {
override def globStatus(pathPattern: Path): Array[FileStatus] = {
mockGlobResults.getOrElse(pathPattern, Array())
}

override def getUri: URI = URI.create("mockFs://mockFs/")
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

is this change needed?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yes, if we don't override this method, then path check inside fs.makeQualified(path) will fail, because it expects path with file:// scheme (MockFileSystem inherits RawLocalFileSystem)

@tigrulya-exe tigrulya-exe force-pushed the SPARK-39910-use-fs-path-qualification branch from 8161581 to 227889b Compare February 8, 2024 08:29
@cloud-fan
Copy link
Contributor

thanks, merging to master/3.5!

@cloud-fan cloud-fan closed this in b7edc5f Feb 8, 2024
cloud-fan pushed a commit that referenced this pull request Feb 8, 2024
…ataSource file path globbing

In current version `DataSource#checkAndGlobPathIfNecessary` qualifies paths via `Path#makeQualified` and `PartitioningAwareFileIndex` qualifies via `FileSystem#makeQualified`. Most `FileSystem` implementations simply delegate to `Path#makeQualified`, but others, like `HarFileSystem` contain fs-specific logic, that can produce different result. Such inconsistencies can lead to a situation, when spark can't find partitions of the source file, because qualified paths, built by `Path` and `FileSystem` are different. Therefore, for uniformity, the `FileSystem` path qualification should be used in `DataSource#checkAndGlobPathIfNecessary`.

Allow users to read files from hadoop archives (.har) using DataFrameReader API

No

New tests were added in `DataSourceSuite` and `DataFrameReaderWriterSuite`

No

Closes #43463 from tigrulya-exe/SPARK-39910-use-fs-path-qualification.

Authored-by: Tigran Manasyan <t.manasyan@arenadata.io>
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
(cherry picked from commit b7edc5f)
Signed-off-by: Wenchen Fan <wenchen@databricks.com>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Projects
None yet
Development

Successfully merging this pull request may close these issues.

3 participants