-
Notifications
You must be signed in to change notification settings - Fork 28.7k
[SPARK-50854][SS] Make path fully qualified before passing it to FileStreamSink #49654
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
9bd12dc
to
c7c16ac
Compare
@dongjoon-hyun @HeartSaVioR Please review |
@dongjoon-hyun Please review |
I'd like to make this be super clear what scenario(s) make us struggle without this fix and how this fix will help resolving it.
I don't think it is very common scenario that people installs Spark in multiple directories and runs driver and executor in separate directory (or any way to set different working directory). Using relative path which resolves to different directories per process doesn't seem like a common scenario and I'd like to see the detail. |
You can explain in other way around - if you see that file source/sink resolves the path in driver in batch query, please describe what's your setup and how you tested and what's the result backing up your claim. It could be used to make a valid claim that we want to have consistence between batch and streaming. |
Please see SPARK-50854. When relative path is used in structured streaming ( This PR does not address the validity and/or use case of the relative path usage that is likely limited to a single node cluster where driver, master and executors have access to the same local file system, though there may be other setups where the same condition applies.
The PR does not target installation into different directories. The problem is reproducible on a single node cluster with default installation (for example dev build or
There is no custom setup done for the Spark. It is default single node cluster. |
While I strongly suspect this is purely based on the setup of the cluster, I'm open for the change to make FileStreamSink work with brew setup. Though I don't like to make any behavior change as it is not justified that this will help any arbitrary setup. Could you please add the sink option in FileStreamSink, like "qualifyRelativePathInDriver", with default "false"? |
I don't see why this is necessary and how it will help as the proposal is not to change the behavior that is incorrect. As the change impacts only relative plan and does not impact cases when absolute plan is specified adding extra option will only add confusion, IMO.
No, it is not specific to the cluster setup. Please try to run the code provided in the JIRA in your cluster setup. |
I've finally found a time to play with this. Sorry, I don't deal with this kind of setup for daily job. I see the default setup from Spark distribution also sets CWD differently between driver and worker. metadata files contain the data file path (despite it's stored outside) correctly, but Spark file reader can't still read the data files properly. I also checked quickly with batch query and saw batch file writer is writing files into the path for driver. I'm yet to read the code path for batch in details, but probably the correction seems to be happening from moving temp file to the final path. I'll take a look at the PR,
Thanks! EDIT: I have thought about this a bit, and it seems to be uneasy to imagine the valid case leveraging prior behavior and seeing it work. I'm OK with the change without flag, but let's just be due diligent via updating migration guide (docs/streaming/ss-migration-guide.md), under Spark 4.0. If we couldn't make it sooner like before Feb 15, this fix will target to Spark 4.1. |
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
Show resolved
Hide resolved
sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala
Outdated
Show resolved
Hide resolved
@HeartSaVioR please take a look |
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.
+1 pending CI
Thanks! Merging to master/4.0. |
…StreamSink ### What changes were proposed in this pull request? 1. Ensure that if relative path is used in `DataStreamWriter`, the path resolution is done on the Spark Driver and is not deferred to Spark Executor. 2. Construct fully qualified path in `DataSource` similar to how it is done for `DataFrameWriter` before it is passed to `FileStreamSink`. 3. Add a check to `FileStreamSink` that asserts that `path` is an absolute path. https://lists.apache.org/thread/ffzwn1y2fgyjw0j09cv4np9z00wymxwv ### Why are the changes needed? To properly support relative paths in structured streaming. The use case mostly applies to single node local Spark cluster. ### Does this PR introduce _any_ user-facing change? The change is only applicable to the use case when relative path is used in `DataStreamWriter`, resulting in data being output to correct directory. No changes are expected for absolute path (the most common production case). ### How was this patch tested? Added new test case to `FileStreamSinkSuite`. ### Was this patch authored or co-authored using generative AI tooling? No Closes #49654 from vrozov/SPARK-50854. Authored-by: Vlad Rozov <[email protected]> Signed-off-by: Jungtaek Lim <[email protected]> (cherry picked from commit 16784fa) Signed-off-by: Jungtaek Lim <[email protected]>
What changes were proposed in this pull request?
DataStreamWriter
, the path resolution is done on the Spark Driver and is not deferred to Spark Executor.DataSource
similar to how it is done forDataFrameWriter
before it is passed toFileStreamSink
.FileStreamSink
that asserts thatpath
is an absolute path.https://lists.apache.org/thread/ffzwn1y2fgyjw0j09cv4np9z00wymxwv
Why are the changes needed?
To properly support relative paths in structured streaming. The use case mostly applies to single node local Spark cluster.
Does this PR introduce any user-facing change?
The change is only applicable to the use case when relative path is used in
DataStreamWriter
, resulting in data being output to correct directory. No changes are expected for absolute path (the most common production case).How was this patch tested?
Added new test case to
FileStreamSinkSuite
.Was this patch authored or co-authored using generative AI tooling?
No