-
Notifications
You must be signed in to change notification settings - Fork 29k
[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
[SPARK-14993] [SQL] Fix Partition Discovery Inconsistency when Input is a Path to Parquet File #12828
Changes from 3 commits
01e4cdf
6835704
9180687
b38a21e
d2b84af
fda8025
ac0dccd
6e0018b
0546772
b37a64f
c2a872c
ab6dbd7
4276356
2dab708
0458770
1debdfa
763706d
4de6ec1
9422a4f
52bdf48
1e95df3
fab24cf
8b2e33b
2ee1876
b9f0090
ade6f7e
9fd63d2
5199d49
404214c
c001dd9
59daa48
41d5f64
472a6e3
0fba10a
cbf73b3
c08f561
474df88
3d9828d
72d2361
07afea5
8bf2007
87a165b
b9359cd
65bd090
babf2da
9e09469
50a8e4a
f3337fa
09cc36d
83a1915
0483145
236a5f4
08aaa4d
64f704e
006ea2d
0c0dc8a
7c4b2f0
38f3af9
461441c
b230e33
bf98150
3ebaf73
252065c
8089c6f
a6c7518
546c1db
e92e9b2
File filter
Filter by extension
Conversations
Jump to
Diff view
Diff view
There are no files selected for viewing
| Original file line number | Diff line number | Diff line change |
|---|---|---|
|
|
@@ -423,23 +423,34 @@ class HDFSFileCatalog( | |
| /** | ||
| * 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. | ||
| * For example, if a user uses `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`. If users want to override the basePath. They can set `basePath` in the options | ||
| * to pass the new base path to the data source. | ||
| * For the above example, if the user-provided base path is `/path/`, the returned | ||
| * base path. | ||
| * | ||
| * By default, the paths of the dataset provided by users will be base paths. | ||
| * Below are three typical examples, | ||
| * 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`. | ||
| */ | ||
| private def basePaths: Set[Path] = { | ||
| val userDefinedBasePath = parameters.get("basePath").map(basePath => Set(new Path(basePath))) | ||
| userDefinedBasePath.getOrElse { | ||
| // If the user does not provide basePath, we will just use paths. | ||
| paths.toSet | ||
| }.map { hdfsPath => | ||
| // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). | ||
| val fs = hdfsPath.getFileSystem(hadoopConf) | ||
| hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) | ||
| parameters.get("basePath").map(new Path(_)) match { | ||
| case Some(userDefinedBasePath) => | ||
| val fs = userDefinedBasePath.getFileSystem(hadoopConf) | ||
| if (!fs.isDirectory(userDefinedBasePath)) { | ||
| throw new IllegalArgumentException("Option 'basePath' must be a directory") | ||
| } | ||
| 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.
I am wondering why I did not just call
makeQualified(fs)when I wrote this part...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 was wondering the same multiple times in other PRs.
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 change it to
fs.makeQualified(userDefinedBasePath). Thanks!