You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by mridulm <gi...@git.apache.org> on 2017/10/13 00:28:52 UTC

[GitHub] spark pull request #19487: [SPARK-21549][CORE] Respect OutputFormats with no...

GitHub user mridulm opened a pull request:

    https://github.com/apache/spark/pull/19487

    [SPARK-21549][CORE] Respect OutputFormats with no/invalid output directory 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
    
    ## How was this patch tested?
    
    Enhanced test to cover new support added.


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/mridulm/spark master

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/19487.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #19487
    
----
commit b15ebe4b6721f9533150aa5831986bea081843e2
Author: Mridul Muralidharan <mr...@gmail.com>
Date:   2017-10-13T00:24:56Z

    Fix HadoopMapReduceCommitProtocol based on failures seen with Phoenix output format

----


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/82709/
    Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    Specifically, this break phoenix output format - where path is set to "".
    This used to work in 2.1 and fails with 2.2 and master.
    
    +CC @szhem, @jiangxb1987, @HyukjinKwon 


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    **[Test build #82709 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82709/testReport)** for PR 19487 at commit [`b15ebe4`](https://github.com/apache/spark/commit/b15ebe4b6721f9533150aa5831986bea081843e2).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by steveloughran <gi...@git.apache.org>.
Github user steveloughran commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    LGTM. I'm going stick out today a slight roll of my PathOutputCommitter class which is one layer above FileOutputCommitter : lets people write committers without output & work paths, yet avoid going near complexity that is FileOutputFormat. See [PathOutputCommitter](https://github.com/hortonworks-spark/cloud-integration/blob/master/spark-cloud-integration/src/main/scala/com/hortonworks/spark/cloud/commit/PathOutputCommitProtocol.scala); if I get my changes into Hadoop 3 then this committer will work for all versions of Hadoop 3.x, even though the S3A stuff is targeting Hadoop 3.1


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19487: [SPARK-21549][CORE] Respect OutputFormats with no...

Posted by steveloughran <gi...@git.apache.org>.
Github user steveloughran commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19487#discussion_r144545827
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala ---
    @@ -60,15 +71,6 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
        */
       private def absPathStagingDir: Path = new Path(path, "_temporary-" + jobId)
    --- End diff --
    
    what if path is null here when passed in? I'd actually expect an IllegalArgumentException being raised, at least from looking at Path.checkParthArg()


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19487: [SPARK-21549][CORE] Respect OutputFormats with no...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19487#discussion_r144580099
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala ---
    @@ -48,6 +49,16 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
       @transient private var committer: OutputCommitter = _
     
       /**
    +   * Checks whether there are files to be committed to a valid output location.
    +   *
    +   * As committing and aborting a job occurs on driver, where `addedAbsPathFiles` is always null,
    +   * it is necessary to check whether a valid output path is specified.
    +   * [[HadoopMapReduceCommitProtocol#path]] need not be a valid [[org.apache.hadoop.fs.Path]] for
    +   * committers not writing to distributed file systems.
    +   */
    +  private val hasValidPath = Try { new Path(path) }.isSuccess
    --- End diff --
    
    I think we should explicitly catch the exception BTW.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19487: [SPARK-21549][CORE] Respect OutputFormats with no...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19487#discussion_r144631350
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala ---
    @@ -60,15 +71,6 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
        */
       private def absPathStagingDir: Path = new Path(path, "_temporary-" + jobId)
    --- End diff --
    
    This should be invoked only if path is valid.
    There is `newTaskTempFileAbsPath` which is not guarded by `hasValidPath` : but that was reviewed in PR #19294 as something which will happen for file output committer (where it is expected to be valid path).
    



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    @HyukjinKwon That was exactly the initial solution I tested locally when we saw the problem with Phoenix.
    The reason to expand it was two fold:
    a) This change preserves existing behavior from 2.1
    b) There are three cases where `new Path` can throw exception, and I would be special case'ing only two of them (null and empty string).
    
    My understanding is that hadoop does not require to create a Path out of `mapred.output.dir`/`mapreduce.output.fileoutputformat.outputdir` - it is upto the individual OutputFormat/Committer to handle it appropriately (unless I am missing something here - @steveloughran would be able to opine better on this).
    
    Expectation of `mapred.output.dir`/`mapreduce.output.fileoutputformat.outputdir` being a valid path is a behavior change we introduced in 2.2 in `HadoopMapReduceCommitProtocol` : since we are now directly handling promotion of files to final location for some cases in `commitJob` or cleanup in `abortJob` : this is done in addition to what is done by committer (which is invoked before our code).
    
    For committers which are not `Path` based, this promotion of output by spark does not apply (there is no path to promote !) - which is what @szhem's patch was fixing - except it was handling only null.
    
    
    Having said all this - if it is the expectation in hadoop that `mapred.output.dir`/`mapreduce.output.fileoutputformat.outputdir` , when specified, should be a valid `Path` - I will hapilly change it to special case'ing only for `null` and `""`.
    Unless we get that clarity, IMO we should preserve behavior and be defensive about when we try to do manual promotion.
    
    Please note that if invalid value is being provided for `mapred.output.dir`/`mapreduce.output.fileoutputformat.outputdir` - the corresponding output format or committer will throw approrpriate exception (like in case of MR or pig or hive).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    Merged build finished. Test PASSed.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    **[Test build #82709 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/82709/testReport)** for PR 19487 at commit [`b15ebe4`](https://github.com/apache/spark/commit/b15ebe4b6721f9533150aa5831986bea081843e2).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19487: [SPARK-21549][CORE] Respect OutputFormats with no...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm closed the pull request at:

    https://github.com/apache/spark/pull/19487


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    > If it does use it, it'll handle an invalid entry in setupJob/setupTask by throwing an exception there.
    
    This should currently happen and `hasValidPath` does not prevent it.
    That is, if committer is unable to handle specified output directory, it can throw exception in `committer.setupJob`; based on what is specified in the config passed in `TaskAttemptContext`.
    
    
    Note that `hasValidPath` and `path` handle the explicit case of absolute path based committer's where `HadoopMapReduceCommitProtocol` moves the result to the final destination (and removes them in case of failure) : see use of `commitJob#taskCommits`.
    `commitJob` does invoke `committer.commitJob` - so committer specific commit will happen.
    This is not relevant for non-path based committer's.
    
    What I would like clarification on is, what is to be done when `path` is invalid.
    My understanding was, this is up to the committer implementation to handle - since it could be a valid use : and if invalid, it would throw an exception in `setupJob` or `commitJob`.
    If this is incorrect assumption, then I will change it back to explicitly support `null` or `""` for `path` - instead of also unparseable path's



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by steveloughran <gi...@git.apache.org>.
Github user steveloughran commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    The more I see of the committer internals, the less confident I am about understanding any of it.
    If your committer isn't writing stuff out, it doesn't need to have any value of mapred.output.dir at all, does it? If it does use it, it'll handle an invalid entry in setupJob/setupTask by throwing an exception there. So the goal of the stuff above it should be to make sure it gets to deal with validating its inputs.
    
    Hadoop trunk adds a new [PathOutputCommitter](https://github.com/steveloughran/hadoop/blob/s3guard/HADOOP-13786-committer/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapreduce/lib/output/PathOutputCommitter.java) class for committers: it's the useful getters of `FileOutputCommitter` pulled up so allowing other committers to provide things like spark the info they need without looking into properties like mapred.output.dir. Have a look at that class and if there is something extra you want pulled up, let me know before Hadoop 3.0 ships & I'll see what I can do
    
    



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #19487: [SPARK-21549][CORE] Respect OutputFormats with no...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on a diff in the pull request:

    https://github.com/apache/spark/pull/19487#discussion_r144633605
  
    --- Diff: core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala ---
    @@ -48,6 +49,16 @@ class HadoopMapReduceCommitProtocol(jobId: String, path: String)
       @transient private var committer: OutputCommitter = _
     
       /**
    +   * Checks whether there are files to be committed to a valid output location.
    +   *
    +   * As committing and aborting a job occurs on driver, where `addedAbsPathFiles` is always null,
    +   * it is necessary to check whether a valid output path is specified.
    +   * [[HadoopMapReduceCommitProtocol#path]] need not be a valid [[org.apache.hadoop.fs.Path]] for
    +   * committers not writing to distributed file systems.
    +   */
    +  private val hasValidPath = Try { new Path(path) }.isSuccess
    --- End diff --
    
    That would depend on whether we want to support invalid paths or not (please see my comment below).
    If we are not supporting invalid paths, I will change this to `null != path && "" != path` explicitly - and have driver throw `IllegalArgumentException` as part of `commitJob` or `abortJob` as earlier.
    
    If we do want to support invalid paths, then exception is not irrelevant : since it indicates an explicit invalid path passed in to output committer (and output committer will suitably log in case parameter is invalid; it is not `HadoopMapReduceCommitProtocol`'s responsibility to do so).


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    I will change from `test:` to `::invalid::` to explicitly indicate an invalid path (I picked the first path which gave me a parse error :) ).
    
    On the question of whether `path` constructor param can be invalid - my understanding @steveloughran was that `mapred.output.dir` conf could be set to any arbitrary string for non path based committers, and it is for the output format and committer to handle it suitably ?
    That is, it need be a valid `Path` and could be encoding some other info ? Hence why I added invalid path handling case - so that we dont keep fighting this `IllegalArgumentException` due to assuming it is a valid path case.
    
    If this understanding is incorrect, I agree with @HyukjinKwon and I will need to make it support only `null` and `""`


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by steveloughran <gi...@git.apache.org>.
Github user steveloughran commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    "" can come in via configuration files; I'd treat that the same as null. Things which aren't valid URIs though, that's something you want to bounce


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    Thanks for cc'ing me @mridulm. Similar concern with ^. 
    
    For `null` case, I thought it makes sense because the property is unset. However, for case of those cases, e.g., `""` and `"test:"`, wouldn't it make sense to not support these because they are invalid values explicitly set?


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    +CC @steveloughran : since you were looking at commit.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by steveloughran <gi...@git.apache.org>.
Github user steveloughran commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    Looking a bit more at this. I see it handles """ as well as empty, and also other forms of invalid URI which Path can't handle today ("multiple colons except with file:// on windows, etc).  And as long as you don't call `absPathStagingDir` you don't get an exception there.
    
    do you actually want the code to downgrade if an invalid URI is passed in, that is: only skip if the path is empty/null, but not for other things like a path of "::::::::::" ? As there you may want to reject it. In which case you'd change the `hasValidPath` query to just looking at the string, and reinstate the test for the path with an invalid non-empty URI
    
     BTW, "test:" is possibly a valid path for "the home directory in the schema 'test'.



---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by mridulm <gi...@git.apache.org>.
Github user mridulm commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    crap, my branch got messed up.
    I will resubmit the PR - sorry for the mess up.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #19487: [SPARK-21549][CORE] Respect OutputFormats with no/invali...

Posted by HyukjinKwon <gi...@git.apache.org>.
Github user HyukjinKwon commented on the issue:

    https://github.com/apache/spark/pull/19487
  
    @mridulm, what do you think about dealing with empty string for now and other cases later if we can't male sure for other cases for now? I guess the actual issue found is about empty string anyway. BTW, we might have to warn about empty string set.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org