-
Notifications
You must be signed in to change notification settings - Fork 28.9k
[SPARK-14993] [SQL] Fix Partition Discovery Inconsistency when Input is a Path to Parquet File #12828
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
Conversation
|
Test build #57500 has finished for PR 12828 at commit
|
|
cc @yhuai |
|
|
||
| if (basePaths.contains(currentPath)) { | ||
| if (basePaths.contains(currentPath) || | ||
| basePaths.exists(_.toString.startsWith(currentPath.toString))) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Can you explain this and provide an example?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I see. We are trying to check if there is a basePath starts with the currentPath.
So, the actual problem is that basePaths in HDFSFileCatalog contains files, right? I discussed it with @tdas. He will have a pr to change basePaths. Let's review his fix together. What do you think?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Sure, please include the test case in
https://github.com/apache/spark/pull/12828/files#diff-cf57fe1c329fb21ac00a8528f049da4aR435
This test case checks three typical cases.
|
@gatorsmile When we call PartitioningUtils.parsePartitions, we should provide a |
|
@yhuai We passed leaf dirs to parsePartition(
path = new Path("file://path/a=10"),
defaultPartitionName = defaultPartitionName,
typeInference = true,
basePaths = Set(new Path("file://path/a=10/p.parquet")))In this case, we need to follow what we did in #9651. The current behavior is shown in the test case: |
|
@yhuai and I discussed that this solution of substring match seems very hacky. The real problem is that basePaths should never have files as it does not make sense to have a basePath that is not a directory. So, our strategy in HDFSFileCatalog of making the set of input files as the default basePath is incorrect. The correct fix is to set the default base path based on the [dirs in input paths] UNION [parent dirs of files in input paths]. Here is the fix - fbef90f Consider updating the scala docs to make this implicit assumption of |
|
@tdas Thank you very much! Will do it soon. |
|
Test build #57599 has finished for PR 12828 at commit
|
| Set(userDefinedBasePath.makeQualified(fs.getUri, fs.getWorkingDirectory)) | ||
|
|
||
| case None => | ||
| paths.map { path => if (leafFiles.contains(path)) path.getParent else path }.toSet |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to make this path qualified?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
leafFiles only contain qualified paths, right?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I believe leaf files contain only qualified. There was comments elsewhere in the file that same so.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Will make path qualified before comparison. Thanks!
|
Sure, I will wait for it. Thanks for letting me know it! |
|
Test build #57711 has finished for PR 12828 at commit
|
# Conflicts: # sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/fileSourceInterfaces.scala
|
Test build #57787 has finished for PR 12828 at commit
|
|
LGTM. Merging to master and branch 2.0 |
…s a Path to Parquet File
#### What changes were proposed in this pull request?
When we load a dataset, if we set the path to ```/path/a=1```, we will not take `a` as the partitioning column. However, if we set the path to ```/path/a=1/file.parquet```, we take `a` as the partitioning column and it shows up in the schema.
This PR is to fix the behavior inconsistency issue.
The base path contains a set of paths that are considered as the base dirs of the input datasets. The partitioning discovery logic will make sure it will stop when it reaches any base path.
By default, the paths of the dataset provided by users will be base paths. Below are three typical cases,
**Case 1**```sqlContext.read.parquet("/path/something=true/")```: the base path will be
`/path/something=true/`, and the returned DataFrame will not contain a column of `something`.
**Case 2**```sqlContext.read.parquet("/path/something=true/a.parquet")```: the base path will be
still `/path/something=true/`, and the returned DataFrame will also not contain a column of
`something`.
**Case 3**```sqlContext.read.parquet("/path/")```: the base path will be `/path/`, and the returned
DataFrame will have the column of `something`.
Users also can override the basePath by setting `basePath` in the options to pass the new base
path to the data source. For example,
```sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/")```,
and the returned DataFrame will have the column of `something`.
The related PRs:
- #9651
- #10211
#### How was this patch tested?
Added a couple of test cases
Author: gatorsmile <[email protected]>
Author: xiaoli <[email protected]>
Author: Xiao Li <[email protected]>
Closes #12828 from gatorsmile/readPartitionedTable.
(cherry picked from commit ef55e46)
Signed-off-by: Yin Huai <[email protected]>
What changes were proposed in this pull request?
When we load a dataset, if we set the path to
/path/a=1, we will not takeaas the partitioning column. However, if we set the path to/path/a=1/file.parquet, we takeaas the partitioning column and it shows up in the schema.This PR is to fix the behavior inconsistency issue.
The base path contains a set of paths that are considered as the base dirs of the input datasets. The partitioning discovery logic will make sure it will stop when it reaches any base path.
By default, the paths of the dataset provided by users will be base paths. Below are three typical cases,
Case 1
sqlContext.read.parquet("/path/something=true/"): the base path will be/path/something=true/, and the returned DataFrame will not contain a column ofsomething.Case 2
sqlContext.read.parquet("/path/something=true/a.parquet"): the base path will bestill
/path/something=true/, and the returned DataFrame will also not contain a column ofsomething.Case 3
sqlContext.read.parquet("/path/"): the base path will be/path/, and the returnedDataFrame will have the column of
something.Users also can override the basePath by setting
basePathin the options to pass the new basepath to the data source. For example,
sqlContext.read.option("basePath", "/path/").parquet("/path/something=true/"),and the returned DataFrame will have the column of
something.The related PRs:
How was this patch tested?
Added a couple of test cases