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

Todd Lipcon commented on MAPREDUCE-5601:
----------------------------------------

Good find.

One question: could we improve this even further by having the client send a 
header like "Max-response-size: <bytes>", and then have the server avoid doing 
any IO for the case where the client is going to abandon the request anyway? 
Seems like we might be incurring extra seeks in some cases due to the behavior 
you described above. It would be unrelated to this JIRA, just thought of it now.

> ShuffleHandler fadvises file regions as DONTNEED even when fetch fails
> ----------------------------------------------------------------------
>
>                 Key: MAPREDUCE-5601
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5601
>             Project: Hadoop Map/Reduce
>          Issue Type: Improvement
>    Affects Versions: 2.2.0
>            Reporter: Sandy Ryza
>            Assignee: Sandy Ryza
>         Attachments: MAPREDUCE-5601.patch, MAPREDUCE-5601.patch
>
>
> When a reducer initiates a fetch request, it does not know whether it will be 
> able to fit the fetched data in memory.  The first part of the response tells 
> how much data will be coming.  If space is not currently available, the 
> reduce will abandon its request and try again later.  When this occurs, the 
> ShuffleHandler still fadvises the file region as DONTNEED.  Meaning that the 
> next time it's asked for, it will definitely be read from disk, even if it 
> happened to be in the page cache before the request.
> I noticed this when trying to figure out why my job was doing so much more 
> disk IO in MR2 than in MR1.  When I turned the fadvise stuff off, I found 
> that disk reads went to nearly 0 on machines that had enough memory to fit 
> map outputs into the page cache.  I then straced the NodeManager and noticed 
> that there were over four times as many fadvise DONTNEED calls as map-reduce 
> pairs.  Further logging showed the same map outputs being fetched about this 
> many times.
> This is a regression from MR1, which only did the fadvise DONTNEED after all 
> the bytes were transferred.



--
This message was sent by Atlassian JIRA
(v6.1#6144)

Reply via email to