[SPARK-49825][SQL] Fix the default warehouse dir for case with white spaces.#54794
Open
fm100 wants to merge 1 commit intoapache:masterfrom
Open
[SPARK-49825][SQL] Fix the default warehouse dir for case with white spaces.#54794fm100 wants to merge 1 commit intoapache:masterfrom
fm100 wants to merge 1 commit intoapache:masterfrom
Conversation
fb7a6d0 to
be4d1ab
Compare
The default warehouse dir, when not set, is the spark-warehouse/ under the current directory. The issue is about when the current directory path contains white space, where the URI resolution makes the path URI encoded. This commit is to fix by using File object directly for the default case. The default is always local file system, and does not need to go through URI resolution. Signed-off-by: Minkyu Park <minkyu.park.200@gmail.com>
be4d1ab to
2b1a18c
Compare
Author
|
@holdenk Thank you for your help. Please review 😄 |
HyukjinKwon
reviewed
Mar 16, 2026
| .version("2.0.0") | ||
| .stringConf | ||
| .createWithDefault(Utils.resolveURI("spark-warehouse").toString) | ||
| .createWithDefault(new File("spark-warehouse").getCanonicalFile.toString) |
Member
There was a problem hiding this comment.
Hm, I think it might be an issue when HDFS scheme is default. new File will always pick the local file as spark-warehouse. Utils.resolveURI("spark-warehouse").toString will result in HDFS's current working directory if default scheme is set as HDFS.
Contributor
There was a problem hiding this comment.
So looking at the resolveURI code path it says:
* Return a well-formed URI for the file described by a user input string.
*
* If the supplied path does not contain a scheme, or is a relative path, it will be
* converted into an absolute path with a file:// scheme.
So it it shouldn't ever give an HDFS path right? Since this is only on the create with default path.
HyukjinKwon
approved these changes
Mar 18, 2026
Member
HyukjinKwon
left a comment
There was a problem hiding this comment.
OK LGTM cc @cloud-fan for a second look
This file contains hidden or bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Sign up for free
to join this conversation on GitHub.
Already have an account?
Sign in to comment
Add this suggestion to a batch that can be applied as a single commit.This suggestion is invalid because no changes were made to the code.Suggestions cannot be applied while the pull request is closed.Suggestions cannot be applied while viewing a subset of changes.Only one suggestion per line can be applied in a batch.Add this suggestion to a batch that can be applied as a single commit.Applying suggestions on deleted lines is not supported.You must change the existing code in this line in order to create a valid suggestion.Outdated suggestions cannot be applied.This suggestion has been applied or marked resolved.Suggestions cannot be applied from pending reviews.Suggestions cannot be applied on multi-line comments.Suggestions cannot be applied while the pull request is queued to merge.Suggestion cannot be applied right now. Please check back later.
What changes were proposed in this pull request?
The default warehouse directory, the value of
spark.sql.warehouse.sqlconf, is alwaysspark-warehouse/under the current directory. The reported issue is for the case when the current directory path contains white spaces. Current behavior of the default conf for warehouse path is to resolve the URI of the current path, which causes the URL encoded string. Because we know that the default warehouse is always the local filesystem directory, this PR proposes to use directly theFileobject instead ofUtils.resolveURIto avoid unnecessary URL encoded string.Why are the changes needed?
Without this PR, when saving dataframe as table like
df.saveAsTable("my_table"), the files for the table is not created under expected$PWD/spark-warehouse/my_table, but it creates a brand new directory that with URL encoded string.Does this PR introduce any user-facing change?
Yes, when the spark process runs at the path with white spaces, the destination of
df.saveAsTablechanges.How was this patch tested?
It was tricky to add a test case since this is related with the process path. Instead I did perform manual testing, making the directory with white spaces, reproduce the issue with the existing code, and then verify the fix. Followed through the reproduce path that the original jira ticket proposed.
Was this patch authored or co-authored using generative AI tooling?
No.