hadoop-mapreduce-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Jian Fang (JIRA)" <j...@apache.org>
Subject [jira] [Commented] (MAPREDUCE-5703) Job client gets failure though RM side job execution result is FINISHED and SUCCEEDED
Date Sat, 16 Aug 2014 00:16:18 GMT

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

Jian Fang commented on MAPREDUCE-5703:
--------------------------------------

We have a cluster with 3 data nodes, but due to some reason, the job history was not persisted
successfully as shown in the AM log.

-------------------------------------------------------------LOG----------------------------------------------------------------------
 ERROR [eventHandlingThread] org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler:
Error writing History Event: org.apache.hadoop.mapreduce.jobhistory.MapAttemptFinishedEvent@1f2cfc93
java.io.IOException: All datanodes 10.253.21.212:9200 are bad. Aborting...
        at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:1140)
        at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.processDatanodeError(DFSOutputStream.java:936)
        at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:491)
----------------------------------------------------------------------------------------------------------------------------------------

As a result, the job failed and threw NPE. The bad thing is the job was marked as failed even
though the job actually finished successfully. 

This means this NPE could happen frequently if the cluster size is not big and it is not an
edge case. 

The method GetTaskAttemptCompletionEventsResponse() fetched a Job by calling verifyAndGetJob(),
but it never checked if job was null or not, which was the root cause of this issue.

    public GetTaskAttemptCompletionEventsResponse
        getTaskAttemptCompletionEvents(
            GetTaskAttemptCompletionEventsRequest request) throws IOException {
      JobId jobId = request.getJobId();
      int fromEventId = request.getFromEventId();
      int maxEvents = request.getMaxEvents();

      Job job = verifyAndGetJob(jobId);
      GetTaskAttemptCompletionEventsResponse response = recordFactory.newRecordInstance(GetTaskAttemptCompletionEventsResponse.class);
      response.addAllCompletionEvents(Arrays.asList(job.getTaskAttemptCompletionEvents(fromEventId,
maxEvents)));
      return response;
    }

Since people may face this problem often for a small cluster, what would be the best way to
fix this issue then? Do retry when save the job to HDFS? Or something else?



> Job client gets failure though RM side job execution result is FINISHED and SUCCEEDED
> -------------------------------------------------------------------------------------
>
>                 Key: MAPREDUCE-5703
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-5703
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>          Components: client
>            Reporter: Ashutosh Jindal
>
> 1) Run MR job 
> 2) After reduce completed and while JHS file writing, restart DN.
> RM side job is shown as successful.
> JHS doesnt have info about the job.
> Job client gets NPE and exit code as 255.
> java.io.IOException: org.apache.hadoop.ipc.RemoteException(java.lang.NullPointerException):
java.lang.NullPointerException
> 	at org.apache.hadoop.mapreduce.v2.hs.HistoryClientService$HSClientProtocolHandler.getTaskAttemptCompletionEvents(HistoryClientService.java:269)
> 	at org.apache.hadoop.mapreduce.v2.api.impl.pb.service.MRClientProtocolPBServiceImpl.getTaskAttemptCompletionEvents(MRClientProtocolPBServiceImpl.java:173)
> 	at org.apache.hadoop.yarn.proto.MRClientProtocol$MRClientProtocolService$2.callBlockingMethod(MRClientProtocol.java:283)
> 	at org.apache.hadoop.ipc.ProtobufRpcEngine$Server$ProtoBufRpcInvoker.call(ProtobufRpcEngine.java:585)
> 	at org.apache.hadoop.ipc.RPC$Server.call(RPC.java:929)
> 	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2080)
> 	at org.apache.hadoop.ipc.Server$Handler$1.run(Server.java:2076)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:396)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1491)
> 	at org.apache.hadoop.ipc.Server$Handler.run(Server.java:2074)
> 	at org.apache.hadoop.mapred.ClientServiceDelegate.invoke(ClientServiceDelegate.java:330)
> 	at org.apache.hadoop.mapred.ClientServiceDelegate.getTaskCompletionEvents(ClientServiceDelegate.java:382)
> 	at org.apache.hadoop.mapred.YARNRunner.getTaskCompletionEvents(YARNRunner.java:529)
> 	at org.apache.hadoop.mapreduce.Job$5.run(Job.java:668)
> 	at org.apache.hadoop.mapreduce.Job$5.run(Job.java:665)
> 	at java.security.AccessController.doPrivileged(Native Method)
> 	at javax.security.auth.Subject.doAs(Subject.java:396)
> 	at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1491)
> 	at org.apache.hadoop.mapreduce.Job.getTaskCompletionEvents(Job.java:665)
> 	at org.apache.hadoop.mapreduce.Job.monitorAndPrintJob(Job.java:1349)
> 	at org.apache.hadoop.mapred.JobClient$NetworkedJob.monitorAndPrintJob(JobClient.java:407)
> 	at org.apache.hadoop.mapred.JobClient.monitorAndPrintJob(JobClient.java:855)
> 	at org.apache.hadoop.mapred.JobClient.runJob(JobClient.java:835)



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Mime
View raw message