-
Notifications
You must be signed in to change notification settings - Fork 28.1k
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-21549][CORE] Respect OutputFormats with no/invalid output directory provided #19497
Conversation
+CC @HyukjinKwon, @steveloughran Sorry for messing up PR #19487 Thanks. |
Test build #82753 has finished for PR 19497 at commit
|
retest this please |
Let me take a look with few tests and be back. Also I think I should cc @jiangxb1987 too. |
Test build #82754 has finished for PR 19497 at commit
|
@mridulm, I just checked through the past related changes and checked the tests pass on branch-2.1. Seems this PR will actually also allow the cases below: .saveAsNewAPIHadoopFile[...]("")
.saveAsNewAPIHadoopFile[...]("::invalid:::") Currently both are failed but looks not after this PR.
I think we should protect these cases. For the cases for old one: .saveAsHadoopFile[...]("")
.saveAsHadoopFile[...]("::invalid:::") these looks failed fast (whether it was initially intended or not) and I guess this PR does not affect these:
|
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.
If we protect the other cases I mentioned above, I think I don't see a reason to block this. Not sure if I see any downside for restoring the previous behaviour back. I am less sure if this fix is the best but looks minimised. So LGTM but @jiangxb1987 I believe It needs your look before we go further.
Thx for taking a deeper look @HyukjinKwon, much appreciated ! |
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.
LGTM, thanks for working on this @mridulm !
…ctory provided ## What changes were proposed in this pull request? PR #19294 added support for null's - but spark 2.1 handled other error cases where path argument can be invalid. Namely: * empty string * URI parse exception while creating Path This is resubmission of PR #19487, which I messed up while updating my repo. ## How was this patch tested? Enhanced test to cover new support added. Author: Mridul Muralidharan <mridul@gmail.com> Closes #19497 from mridulm/master. (cherry picked from commit 13c1559) Signed-off-by: Mridul Muralidharan <mridul@gmail.com>
Thanks for the reviews everyone ! |
@mridulm, BTW, WDYT about disallowing:
within the APIs? If i tested this correctly, this PR also allows both cases but I think we should disallow as it requires |
@HyukjinKwon My intention was to preserve earlier behavior. |
To clarify, we can look at changing the behavior (if required) in future - but that should be an explicit design choice informed by hadoop committer design. Until then, we should look to interoperate. |
I support this PR itself of course. I have no problem with this. I meant a separate (soft) question about |
Do you mean |
I meant
I wanted to talk about this.
. So, I suspected
spark/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala Lines 1004 to 1008 in 3f958a9
spark/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala Lines 983 to 987 in 3f958a9
|
I agree this was focussed more on the regression introduced and it should be good enough already, and I am talking about a different thing for behaviour change. Let me organise my idea and and try to file a JIRA later. I think strictly this is separate anyway if I haven't missed something or simply I was wrong. |
@HyukjinKwon Thanks for clarifying. The way I look at it is: Additionally, in createPathFromString for In contrast Having said that, we can always iterate in a jira if you feel there is some confusion - it is always better to be explicitly clear about the interfaces we expose and support ! |
Thank you @mridulm. I regret that I raised this here, causing confusion. Let's talk more in another place. I will cc you (and @jiangxb1987) when I happened to file up a JIRA or see similar issue related with this. |
I guess one aspect of If handling of paths is to be done in the committer, Looking at how Hadoop's FileOutputFormat implementations work, they can handle a null/undefined output dir property, but not an empty one. public static Path getOutputPath(JobContext job) {
String name = job.getConfiguration().get(FileOutputFormat.OUTDIR);
return name == null ? null: new Path(name); Which implies that |
…ctory provided ## What changes were proposed in this pull request? PR apache#19294 added support for null's - but spark 2.1 handled other error cases where path argument can be invalid. Namely: * empty string * URI parse exception while creating Path This is resubmission of PR apache#19487, which I messed up while updating my repo. ## How was this patch tested? Enhanced test to cover new support added. Author: Mridul Muralidharan <mridul@gmail.com> Closes apache#19497 from mridulm/master. (cherry picked from commit 13c1559) Signed-off-by: Mridul Muralidharan <mridul@gmail.com>
What changes were proposed in this pull request?
PR #19294 added support for null's - but spark 2.1 handled other error cases where path argument can be invalid.
Namely:
This is resubmission of PR #19487, which I messed up while updating my repo.
How was this patch tested?
Enhanced test to cover new support added.