hadoop-mapreduce-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jason Lowe (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (MAPREDUCE-7026) Shuffle Fetcher does not log the actual error message thrown by ShuffleHandler
Date Tue, 24 Apr 2018 15:24:00 GMT

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

Jason Lowe commented on MAPREDUCE-7026:
---------------------------------------

Thanks for the report and the patch!

The error posted from the Fetcher in the description is not what this is targeting -- that's
simply the error message that is printed when the Fetcher meets the maximum failure threshold
and decides to kill the reducer due to lack of sufficient shuffle progress.  This patch will
not affect that error message.

Do you have some sample log output after this patch has been applied?  The IllegalArgumentException
is because the Fetcher is trying to consume a ShuffleHeader when the NM has instead decided
to return error text.  One potential problem with the approach is that the process of trying
to parse a shuffle header could have consumed some of the error message before the error was
thrown (i.e.: whatever bytes the readFields call consumed), and the approach in the patch
will lose those bytes.  It'd be good to see some sample output to see how it's working in
practice.

There's no limit to how much data this will try to buffer.  If there's a corrupted bit in
the shuffle header and the NM is _not_ emitting an error message then this code could attempt
to buffer many megabytes (or gigabytes!) of shuffle data after the shuffle header.  That could
blow the heap and cause a teardown of the task on an error that should be retriable.  There
should be a reasonable limit on the amount of data this will try to consume before an EOF
is reached.  Also attempting to read compressed data looking for a line terminator could cause
us to consume a lot of data before we find one.

Why is StringBuffer being used instead of StringBuilder?  Synchronization is not needed here.
 Also why is an empty string being passed to the constructor rather than calling the default
constructor?


> Shuffle Fetcher does not log the actual error message thrown by ShuffleHandler
> ------------------------------------------------------------------------------
>
>                 Key: MAPREDUCE-7026
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-7026
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: task
>    Affects Versions: 2.7.3
>            Reporter: Prabhu Joseph
>            Assignee: Prabhu Joseph
>            Priority: Major
>              Labels: supportability
>         Attachments: MAPREDUCE-7026.1.patch, MAPREDUCE-7026.2.patch
>
>
> A job is failing with reduce tasks failed to fetch map output and the NodeManager ShuffleHandler
failed to serve the map outputs with some IOException like below. ShuffleHandler sends the
actual error message in response inside sendError() but the Fetcher does not log this message.
> Logs from NodeManager ShuffleHandler:
> {code}
> 2017-12-18 10:10:30,728 ERROR mapred.ShuffleHandler (ShuffleHandler.java:messageReceived(962))
- Shuffle error in populating headers :
> java.io.IOException: Error Reading IndexFile
>         at org.apache.hadoop.mapred.ShuffleHandler$Shuffle.populateHeaders(ShuffleHandler.java:1089)
>         at org.apache.hadoop.mapred.ShuffleHandler$Shuffle.messageReceived(ShuffleHandler.java:958)
>         at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
>         at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:560)
>         at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:787)
>         at org.jboss.netty.handler.stream.ChunkedWriteHandler.handleUpstream(ChunkedWriteHandler.java:142)
>         at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:560)
>         at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:787)
>         at org.jboss.netty.handler.codec.http.HttpChunkAggregator.messageReceived(HttpChunkAggregator.java:148)
>         at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
>         at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:560)
>         at org.jboss.netty.channel.DefaultChannelPipeline$DefaultChannelHandlerContext.sendUpstream(DefaultChannelPipeline.java:787)
>         at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:296)
>         at org.jboss.netty.handler.codec.frame.FrameDecoder.unfoldAndFireMessageReceived(FrameDecoder.java:459)
>         at org.jboss.netty.handler.codec.replay.ReplayingDecoder.callDecode(ReplayingDecoder.java:536)
>         at org.jboss.netty.handler.codec.replay.ReplayingDecoder.messageReceived(ReplayingDecoder.java:435)
>         at org.jboss.netty.channel.SimpleChannelUpstreamHandler.handleUpstream(SimpleChannelUpstreamHandler.java:70)
>         at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:560)
>         at org.jboss.netty.channel.DefaultChannelPipeline.sendUpstream(DefaultChannelPipeline.java:555)
>         at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:268)
>         at org.jboss.netty.channel.Channels.fireMessageReceived(Channels.java:255)
>         at org.jboss.netty.channel.socket.nio.NioWorker.read(NioWorker.java:88)
>         at org.jboss.netty.channel.socket.nio.AbstractNioWorker.process(AbstractNioWorker.java:107)
>         at org.jboss.netty.channel.socket.nio.AbstractNioSelector.run(AbstractNioSelector.java:312)
>         at org.jboss.netty.channel.socket.nio.AbstractNioWorker.run(AbstractNioWorker.java:88)
>         at org.jboss.netty.channel.socket.nio.NioWorker.run(NioWorker.java:178)
>         at org.jboss.netty.util.ThreadRenamingRunnable.run(ThreadRenamingRunnable.java:108)
>         at org.jboss.netty.util.internal.DeadLockProofWorker$1.run(DeadLockProofWorker.java:42)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>         at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: Owner 'hbase' for path /grid/7/hadoop/yarn/local/usercache/bde/appcache/application_1512457770852_9447/output/attempt_1512457770852_9447_1_01_000007_0_10004/file.out.index
did not match expected owner 'bde'
>         at org.apache.hadoop.io.SecureIOUtils.checkStat(SecureIOUtils.java:285)
>         at org.apache.hadoop.io.SecureIOUtils.forceSecureOpenFSDataInputStream(SecureIOUtils.java:174)
>         at org.apache.hadoop.io.SecureIOUtils.openFSDataInputStream(SecureIOUtils.java:158)
>         at org.apache.hadoop.mapred.SpillRecord.<init>(SpillRecord.java:70)
>         at org.apache.hadoop.mapred.SpillRecord.<init>(SpillRecord.java:62)
>         at org.apache.hadoop.mapred.IndexCache.readIndexFileToCache(IndexCache.java:119)
> {code}
> Fetcher Logs below without the actual error message:
> {code}
> 2017-12-18 10:10:17,688 INFO [IPC Server handler 1 on 35118] org.apache.hadoop.mapred.TaskAttemptListenerImpl:
Diagnostics report from attempt_1511248592679_0039_r_000000_0: Error: org.apache.hadoop.mapreduce.task.reduce.Shuffle$ShuffleError:
error in shuffle in fetcher#3
>         at org.apache.hadoop.mapreduce.task.reduce.Shuffle.run(Shuffle.java:134)
>         at org.apache.hadoop.mapred.ReduceTask.run(ReduceTask.java:376)
>         at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:170)
>         at java.security.AccessController.doPrivileged(Native Method)
>         at javax.security.auth.Subject.doAs(Subject.java:422)
>         at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1866)
>         at org.apache.hadoop.mapred.YarnChild.main(YarnChild.java:164)
> Caused by: java.io.IOException: Exceeded MAX_FAILED_UNIQUE_FETCHES; bailing-out.
>         at org.apache.hadoop.mapreduce.task.reduce.ShuffleSchedulerImpl.checkReducerHealth(ShuffleSchedulerImpl.java:366)
>         at org.apache.hadoop.mapreduce.task.reduce.ShuffleSchedulerImpl.copyFailed(ShuffleSchedulerImpl.java:288)
>         at org.apache.hadoop.mapreduce.task.reduce.Fetcher.copyFromHost(Fetcher.java:354)
>         at org.apache.hadoop.mapreduce.task.reduce.Fetcher.run(Fetcher.java:193)
> {code}



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

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


Mime
View raw message