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

Jason Lowe commented on YARN-2410:
----------------------------------

Thanks for the patch, Kuhu!

Do we really need getShuffle/setShuffle and a new shuffle field?  Much simpler 
to access pipelineFact.SHUFFLE directly or add a getShuffle method to 
HttpPipelineFactory and use that.  We shouldn't be redundantly tracking shuffle 
in the ShuffleHandler class.

Rather than catch and swallowing the exception from metrics.operationComplete 
(which shouldn't happen), can't we just let this propagate up?  I would think 
it would eventually trigger exceptionCaught to be called which should do the 
right thing like any other exception on the channel.

The ChannelHandlerContext has the corresponding channel, so do we really need 
SendMapOutputParams to store it separately?

SendMapOutputParams should be treated like an immutable object.  There's no 
need to set any of its contents after it is created, so we should just remove 
all the set methods.  Same applies to ReduceContext.  Actually I'm not really 
sure why SendMapOutputParams exists separate from ReduceContext.  There should 
be a one-to-one relationship there.  I think we could just promote all of the 
members of SendMapOutputParams into ReduceContext and cut down on some of the 
excess boilerplate.

ReduceContext has an unused Configuration variable and getConf is never called.

Seems like we could associate the ReduceContext with the ReduceMapFileCount 
listener object directly when we construct it.  That way we don't have to fish 
it out of the channel attachment and the code plays nicer with other things 
that might want to use the channel attachment for something.

It looks like we can now send an INTERNAL_SERVER_ERROR followed by a NOT_FOUND 
error since sendMap will return null when it sends an internal error.  Maybe 
sendMap should take care of sending all the appropriate errors directly.

Why was reduceContext added as a TestShuffleHandler instance variable?  It's 
specific to the new test.

The test would be a bit more readable with some factoring out of some code to 
utility methods, e.g.: createMockChannel method which takes care of mocking up 
all the stuff needed to mock a channel.  Also the overridden ShuffleHandler is 
enough code that it should just be a separate class within the test rather than 
inline in the method.

Nit: The comment for mapreduce.shuffle.max.send.map.count should mention 
simultaneous or concurrent otherwise it implies it will only send that many 
outputs total.  Also it may be more clear if the property were named something 
like mapreduce.shuffle.max.session-open-files or something similar, although 
I'm not super excited about that name either.

Nit: some (all?) of the returns in ReduceMapFileCount.operationComplete would 
be easier to follow using an {{else}} clause, e.g.:
{code}
      if (waitCount == 0) {
        metrics.operationComplete(future);
        future.getChannel().close();
      } else {
        shuffle.sendMap(rc.getSendMapOutputParams().getCtx(),
            rc.getSendMapOutputParams().getInfoMap());
      }
{code}

Nit: whitespace between field definitions and internal class definitions and 
also between method definitions would help readability.

Nit: Per the coding conventions class and instance variables should appear 
before constructors which in turn appear before methods

Nit: variables should be declared when they are initialized if possible, e.g.: 
nextId and mapId in sendMap.

Nit: Please organize the imports in the test, there's a mix of static and 
non-static imports.


> Nodemanager ShuffleHandler can possible exhaust file descriptors
> ----------------------------------------------------------------
>
>                 Key: YARN-2410
>                 URL: https://issues.apache.org/jira/browse/YARN-2410
>             Project: Hadoop YARN
>          Issue Type: Bug
>          Components: nodemanager
>    Affects Versions: 2.5.0
>            Reporter: Nathan Roberts
>            Assignee: Kuhu Shukla
>         Attachments: YARN-2410-v1.patch, YARN-2410-v2.patch, 
> YARN-2410-v3.patch, YARN-2410-v4.patch, YARN-2410-v5.patch
>
>
> The async nature of the shufflehandler can cause it to open a huge number of
> file descriptors, when it runs out it crashes.
> Scenario:
> Job with 6K reduces, slow start set to 0.95, about 40 map outputs per node.
> Let's say all 6K reduces hit a node at about same time asking for their
> outputs. Each reducer will ask for all 40 map outputs over a single socket in 
> a
> single request (not necessarily all 40 at once, but with coalescing it is
> likely to be a large number).
> sendMapOutput() will open the file for random reading and then perform an 
> async transfer of the particular portion of this file(). This will 
> theoretically
> happen 6000*40=240000 times which will run the NM out of file descriptors and 
> cause it to crash.
> The algorithm should be refactored a little to not open the fds until they're
> actually needed. 



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

Reply via email to