Skip to content
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

Closed
wants to merge 1 commit into from

Conversation

mridulm
Copy link
Contributor

@mridulm mridulm commented Oct 14, 2017

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.

@mridulm
Copy link
Contributor Author

mridulm commented Oct 14, 2017

+CC @HyukjinKwon, @steveloughran

Sorry for messing up PR #19487
The only change in this PR is to use ::invalid:: instead of test: in the test to address @steveloughran's comment.

Thanks.

@SparkQA
Copy link

SparkQA commented Oct 14, 2017

Test build #82753 has finished for PR 19497 at commit a319df3.

  • This patch fails due to an unknown error code, -9.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

retest this please

@HyukjinKwon
Copy link
Member

Let me take a look with few tests and be back. Also I think I should cc @jiangxb1987 too.

@SparkQA
Copy link

SparkQA commented Oct 14, 2017

Test build #82754 has finished for PR 19497 at commit a319df3.

  • This patch passes all tests.
  • This patch merges cleanly.
  • This patch adds no public classes.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Oct 15, 2017

@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.

Can not create a Path from an empty string
java.lang.IllegalArgumentException: Can not create a Path from an empty string
	at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
	at org.apache.hadoop.fs.Path.<init>(Path.java:135)
	at org.apache.hadoop.fs.Path.<init>(Path.java:89)
	at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.absPathStagingDir(HadoopMapReduceCommitProtocol.scala:61)
...
java.net.URISyntaxException: Relative path in absolute URI: ::invalid:::
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: ::invalid:::
	at org.apache.hadoop.fs.Path.initialize(Path.java:206)
	at org.apache.hadoop.fs.Path.<init>(Path.java:172)
	at org.apache.hadoop.fs.Path.<init>(Path.java:89)
	at org.apache.spark.internal.io.HadoopMapReduceCommitProtocol.absPathStagingDir(HadoopMapReduceCommitProtocol.scala:61)
...

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:

Can not create a Path from an empty string
java.lang.IllegalArgumentException: Can not create a Path from an empty string
	at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
	at org.apache.hadoop.fs.Path.<init>(Path.java:135)
	at org.apache.spark.internal.io.SparkHadoopWriterUtils$.createPathFromString(SparkHadoopWriterUtils.scala:54)
java.net.URISyntaxException: Relative path in absolute URI: ::invalid:::
java.lang.IllegalArgumentException: java.net.URISyntaxException: Relative path in absolute URI: ::invalid:::
	at org.apache.hadoop.fs.Path.initialize(Path.java:206)
	at org.apache.hadoop.fs.Path.<init>(Path.java:172)
	at org.apache.spark.internal.io.SparkHadoopWriterUtils$.createPathFromString(SparkHadoopWriterUtils.scala:54)

Copy link
Member

@HyukjinKwon HyukjinKwon left a 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.

@mridulm
Copy link
Contributor Author

mridulm commented Oct 15, 2017

Thx for taking a deeper look @HyukjinKwon, much appreciated !
I will wait for @jiangxb1987 to also opine before committing - I want to make sure we are not adding incorrect behavior; given that this is a followup to an earlier PR (some excellent work by @szhem btw)

Copy link
Contributor

@jiangxb1987 jiangxb1987 left a 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 !

asfgit pushed a commit that referenced this pull request Oct 16, 2017
…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>
@asfgit asfgit closed this in 13c1559 Oct 16, 2017
@mridulm
Copy link
Contributor Author

mridulm commented Oct 16, 2017

Thanks for the reviews everyone !

@HyukjinKwon
Copy link
Member

@mridulm, BTW, WDYT about disallowing:

.saveAsNewAPIHadoopFile[...]("")
.saveAsNewAPIHadoopFile[...]("::invalid:::")

within the APIs? If i tested this correctly, this PR also allows both cases but I think we should disallow as it requires path and overrides it and saveAsHadoopFile disallows it. Seems this is also allowed in branch-2.1 as I recall correctly but looks disallowing it sounds more making sense in any event.

@mridulm
Copy link
Contributor Author

mridulm commented Oct 16, 2017

@HyukjinKwon My intention was to preserve earlier behavior.
Particularly for non-path based committer's, the path variable and its use/processing is not relevant, it makes more sense to ignore that codepath entirely.

@mridulm
Copy link
Contributor Author

mridulm commented Oct 16, 2017

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.

@HyukjinKwon
Copy link
Member

I support this PR itself of course. I have no problem with this.

I meant a separate (soft) question about saveAsNewAPIHadoopFile (not saveAsNewAPIHadoopDataset) to validate path parameter which we take in saveAsNewAPIHadoopFile explicitly.

@mridulm
Copy link
Contributor Author

mridulm commented Oct 16, 2017

saveAsNewAPIHadoopFile simply delegates to saveAsNewAPIHadoopDataset (with some options set), right ? The behavior would be similar ?

Do you mean saveAsHadoopDataset instead ?
I did not change behavior there - since the exception was getting raised from within hadoop code and not from our code (when we pass invalid values), and it is preserving behavior from earlier code.
I was focussed more on the regression introduced.

@HyukjinKwon
Copy link
Member

HyukjinKwon commented Oct 16, 2017

I meant saveAsNewAPIHadoopFile comparing to saveAsHadoopFile.

saveAsNewAPIHadoopFile[...]("") // succeeds
saveAsHadoopFile[...]("") // fails

Can not create a Path from an empty string
java.lang.IllegalArgumentException: Can not create a Path from an empty string
	at org.apache.hadoop.fs.Path.checkPathArg(Path.java:127)
	at org.apache.hadoop.fs.Path.<init>(Path.java:135)
	at org.apache.spark.internal.io.SparkHadoopWriterUtils$.createPathFromString(SparkHadoopWriterUtils.scala:54)
	at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopFile$4.apply$mcV$sp(PairRDDFunctions.scala:1066)
	at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopFile$4.apply(PairRDDFunctions.scala:1032)
	at org.apache.spark.rdd.PairRDDFunctions$$anonfun$saveAsHadoopFile$4.apply(PairRDDFunctions.scala:1032)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151)
	at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112)
	at org.apache.spark.rdd.RDD.withScope(RDD.scala:363)
	at org.apache.spark.rdd.PairRDDFunctions.saveAsHadoopFile(PairRDDFunctions.scala:1032)

I wanted to talk about this. saveAsHadoopFile seems being failed fast within saveAsHadoopFile specifically before the delegation:

SparkHadoopWriterUtils.createPathFromString(path, hadoopConf))

. So, I suspected saveAsNewAPIHadoopFile should also throw an exception in this way.

saveAsHadoopFile validates path so I thought saveAsNewAPIHadoopFile should also validate.

* Output the RDD to any Hadoop-supported file system, using a Hadoop `OutputFormat` class
* supporting the key and value types K and V in this RDD. Compress with the supplied codec.
*/
def saveAsHadoopFile(
path: String,

* Output the RDD to any Hadoop-supported file system, using a new Hadoop API `OutputFormat`
* (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD.
*/
def saveAsNewAPIHadoopFile(
path: String,

@HyukjinKwon
Copy link
Member

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.

@mridulm
Copy link
Contributor Author

mridulm commented Oct 17, 2017

@HyukjinKwon Thanks for clarifying.

The way I look at it is:
saveAsHadoopFile is explicitly referring to Output the RDD to any Hadoop-supported file system in its description (and name) - and so valid Path is a reasonable requirement.

Additionally, in createPathFromString for path == null we are explicitly throwing IllegalArgumentException (new Path will do the same now, but I think this changed in past where it used to result in NPE ?).
The subsequent val outputPath = new Path(path) will do that for other invalid input paths as well.

In contrast saveAsHadoopDataset is not related to file system but Output the RDD to any Hadoop-supported storage system : where output being a valid Path is not a requirement.

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 !
Thanks.

@HyukjinKwon
Copy link
Member

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.

@steveloughran
Copy link
Contributor

I guess one aspect of saveAsNewAPIHadoopFile is that it calls jobConfiguration.set("mapreduce.output.fileoutputformat.outputdir", path), and Configuration.set(String key, String value) has a check for null key or value.

If handling of paths is to be done in the committer, saveAsNewAPIHadoopFile should really be looking @ path and calling jobConfiguration.unset("mapreduce.output.fileoutputformat.outputdir) if path==null.

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 saveAsNewHadoopFile("") might want to unset the config option too, so offloading the problem of what happens on an empty path to the committer. Though I'd recommend checking to see what meaningful exceptions actually get raised in this situation when the committer is the normal FileOutputFormat/FileOutputCommitter setup

MatthewRBruce pushed a commit to Shopify/spark that referenced this pull request Jul 31, 2018
…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>
Sign up for free to join this conversation on GitHub. Already have an account? Sign in to comment
Labels
None yet
Projects
None yet
5 participants