You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-issues@hadoop.apache.org by "Daniel Templeton (JIRA)" <ji...@apache.org> on 2016/09/21 17:50:20 UTC

[jira] [Commented] (MAPREDUCE-6728) Give fetchers hint when ShuffleHandler rejects a shuffling connection

    [ https://issues.apache.org/jira/browse/MAPREDUCE-6728?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15510648#comment-15510648 ] 

Daniel Templeton commented on MAPREDUCE-6728:
---------------------------------------------

Thanks for the patch, [~haibochen].

Comments:
* Instead of reusing the {{INITIAL_DELAY}}, you should define a retry delay instead.  You might also want to consider some kind of backoff.  The most correct approach would be to define the delay in the {{ShuffleHandler}} and pass it back in the {{Retry-after}} header.
* I don't love defining an inner exception, but it appears to be the best option.  Can we call it something like {{TryAgainLaterException}} so that it's really clear what it means?  Should it be _static_?  It should probably be _private_.
* In this line the whitespace is wrong: {code}for(TaskAttemptID left: remaining) {{code}
* Is there a clever way to not duplicate the code to put back the remaining attempts? It appears in both _catch_ clauses.
* Please include javadoc for your {{ShuffleSchedulerImpl.penalize()}} method.
* In the {{ShuffleHandler.Shuffle.channelOpen()}} method, instead of writing the error code there, you should probably call {{sendError()}}.
* In the {{TestFetcher}} test, watching the calls to {{hostFailed}} and {{copyFailed}} seems brittle.  Maybe instead watch the {{ioErrs}} counter?
* Please include a message in your new assert in {{TestShuffleHandler}}.

> Give fetchers hint when ShuffleHandler rejects a shuffling connection
> ---------------------------------------------------------------------
>
>                 Key: MAPREDUCE-6728
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-6728
>             Project: Hadoop Map/Reduce
>          Issue Type: Improvement
>          Components: mrv2
>            Reporter: Haibo Chen
>            Assignee: Haibo Chen
>         Attachments: mapreduce6728.001.patch, mapreduce6728.prelim.patch
>
>
> If # of open shuffle connection to a node goes over the max, ShuffleHandler closes the connection immediately without giving fetchers any hint of the reason, which causes fetchers to fail due to exceptions 
> java.net.SocketException: Unexpected end of file from server
> 	at sun.net.www.http.HttpClient.parseHTTPHeader(HttpClient.java:772)
> 	at sun.net.www.http.HttpClient.parseHTTP(HttpClient.java:633)
> 	at sun.net.www.http.HttpClient.parseHTTPHeader(HttpClient.java:769)
> 	at sun.net.www.http.HttpClient.parseHTTP(HttpClient.java:633)
> 	at sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1323)
> 	at java.net.HttpURLConnection.getResponseCode(HttpURLConnection.java:468)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.verifyConnection(Fetcher.java:430)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.setupConnectionsWithRetry(Fetcher.java:395)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.openShuffleUrl(Fetcher.java:266)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.copyFromHost(Fetcher.java:323)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.run(Fetcher.java:193)
> OR 
> java.net.SocketException: Connection reset
> 	at java.net.SocketInputStream.read(SocketInputStream.java:196)
> 	at java.net.SocketInputStream.read(SocketInputStream.java:122)
> 	at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
> 	at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
> 	at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
> 	at sun.net.www.http.HttpClient.parseHTTPHeader(HttpClient.java:687)
> 	at sun.net.www.http.HttpClient.parseHTTP(HttpClient.java:633)
> 	at sun.net.www.http.HttpClient.parseHTTPHeader(HttpClient.java:769)
> 	at sun.net.www.http.HttpClient.parseHTTP(HttpClient.java:633)
> 	at sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1323)
> 	at java.net.HttpURLConnection.getResponseCode(HttpURLConnection.java:468)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.verifyConnection(Fetcher.java:430)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.setupConnectionsWithRetry(Fetcher.java:395)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.openShuffleUrl(Fetcher.java:266)
> 	at org.apache.hadoop.mapreduce.task.reduce.Fetcher.copyFromHost(Fetcher.java
> Such failures are counted as fetcher failures



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
To unsubscribe, e-mail: mapreduce-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: mapreduce-issues-help@hadoop.apache.org