Morty Zhong created MAPREDUCE-7176:
--------------------------------------

             Summary: ShuffleMetrics not count when sendMap future 
unsuccessfully
                 Key: MAPREDUCE-7176
                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7176
             Project: Hadoop Map/Reduce
          Issue Type: Bug
          Components: mrv2
    Affects Versions: 2.9.2
            Reporter: Morty Zhong


ShuffleMetrics count for shuffle metrics in NM(shuffle failed/OK or current 
connections).

But the condition that invoke ShuffleMetrics to do counting is when reduceMap 
ChannelFuture is successful.

so when net I/O error or other error, ShuffleMetrics won`t count. 

following are the code(comments start with '###' are added by me)

ReduceMapFileCount.class(where invoke ShuffleMetrics)
{code:java}
public void operationComplete(ChannelFuture future) throws Exception {
  if (!future.isSuccess()) { // #### return directly when unsuccessful
    future.getChannel().close();
    return;
  }
  int waitCount = this.reduceContext.getMapsToWait().decrementAndGet();
  if (waitCount == 0) {
    metrics.operationComplete(future);
    // Let the idle timer handler close keep-alive connections
    if (reduceContext.getKeepAlive()) {
      ChannelPipeline pipeline = future.getChannel().getPipeline();
      TimeoutHandler timeoutHandler =
          (TimeoutHandler)pipeline.get(TIMEOUT_HANDLER);
      timeoutHandler.setEnabledTimeout(true);
    } else {
      future.getChannel().close();
    }
  } else {
    pipelineFact.getSHUFFLE().sendMap(reduceContext);
  }
}
{code}
ShuffleMetrics:
{code:java}
public void operationComplete(ChannelFuture future) throws Exception {
  if (future.isSuccess()) {
    shuffleOutputsOK.incr();
  } else {
    shuffleOutputsFailed.incr(); //### never be invoked
  }
  shuffleConnections.decr(); // ### some conditions won`t be invoked
}
{code}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

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

Reply via email to