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

Junping Du commented on MAPREDUCE-6654:
---------------------------------------

Thanks [~xiaochen] for reporting this issue. We run into the similar issue 
recently. The root cause is different (NN in safe mode) as exception blow:
{noformat}
The reported blocks 2808 has reached the threshold 0.9900 of total blocks 2808. 
The number of live datanodes 20 has reached the minimum number 0. In safe mode 
extension. Safe mode will be turned off automatically in 1 seconds.
        at 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.checkNameNodeSafeMode(FSNamesystem.java:1359)
        ... 13 more
        at org.apache.hadoop.ipc.Client.call(Client.java:1426)
        at org.apache.hadoop.ipc.Client.call(Client.java:1363)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:229)
        at com.sun.proxy.$Proxy10.create(Unknown Source)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.create(ClientNamenodeProtocolTranslatorPB.java:298)
        at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
        at 
sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)
        at 
sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
        at java.lang.reflect.Method.invoke(Method.java:606)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:256)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:104)
        at com.sun.proxy.$Proxy11.create(Unknown Source)
        at 
org.apache.hadoop.hdfs.DFSOutputStream.newStreamForCreate(DFSOutputStream.java:1716)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1703)
        at org.apache.hadoop.hdfs.DFSClient.create(DFSClient.java:1638)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:449)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem$7.doCall(DistributedFileSystem.java:445)
        at 
org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:445)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:388)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:909)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:890)
        at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:787)
        at 
org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.createEventWriter(JobHistoryEventHandler.java:421)
        at 
org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.setupEventWriter(JobHistoryEventHandler.java:461)
        at 
org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.handleEvent(JobHistoryEventHandler.java:551)
        at 
org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler$1.run(JobHistoryEventHandler.java:320)
{noformat}

But it hits to the same issue - as the first historic event, AM_STARTED event 
doesn't setup Event Writer for this job successfully. And the solution should 
be: for any type of event, when mi is null, try to setup event writer again or 
throw ioe if not succeed, so NPE get throw.

Assign to myself and will put on a patch soon.


> Possible NPE in JobHistoryEventHandler#handleEvent
> --------------------------------------------------
>
>                 Key: MAPREDUCE-6654
>                 URL: https://issues.apache.org/jira/browse/MAPREDUCE-6654
>             Project: Hadoop Map/Reduce
>          Issue Type: Bug
>            Reporter: Xiao Chen
>            Assignee: Junping Du
>            Priority: Minor
>
> I have seen NPE thrown from {{JobHistoryEventHandler#handleEvent}}:
> {noformat}
> 2016-03-14 16:42:15,231 INFO [Thread-69] 
> org.apache.hadoop.service.AbstractService: Service JobHistoryEventHandler 
> failed in state STOPPED; cause: java.lang.NullPointerException
> java.lang.NullPointerException
>       at 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.handleEvent(JobHistoryEventHandler.java:570)
>       at 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.serviceStop(JobHistoryEventHandler.java:382)
>       at 
> org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
>       at 
> org.apache.hadoop.service.ServiceOperations.stop(ServiceOperations.java:52)
>       at 
> org.apache.hadoop.service.ServiceOperations.stopQuietly(ServiceOperations.java:80)
>       at 
> org.apache.hadoop.service.CompositeService.stop(CompositeService.java:157)
>       at 
> org.apache.hadoop.service.CompositeService.serviceStop(CompositeService.java:131)
>       at 
> org.apache.hadoop.mapreduce.v2.app.MRAppMaster.serviceStop(MRAppMaster.java:1651)
>       at 
> org.apache.hadoop.service.AbstractService.stop(AbstractService.java:221)
>       at 
> org.apache.hadoop.mapreduce.v2.app.MRAppMaster.stop(MRAppMaster.java:1147)
>       at 
> org.apache.hadoop.mapreduce.v2.app.MRAppMaster.shutDownJob(MRAppMaster.java:573)
>       at 
> org.apache.hadoop.mapreduce.v2.app.MRAppMaster$JobFinishEventHandler$1.run(MRAppMaster.java:620)
> {noformat}
> In the version this exception is thrown, the 
> [line|https://github.com/apache/hadoop/blob/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/jobhistory/JobHistoryEventHandler.java#L586]
>  is:
> {code:java}mi.writeEvent(historyEvent);{code}
> IMHO, this may be caused by an exception in a previous step. Specifically, in 
> the kerberized environment, when creating event writer which calls to decrypt 
> EEK, the connection to KMS failed. Exception below:
> {noformat} 
> 2016-03-14 16:41:57,559 ERROR [eventHandlingThread] 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler: Error 
> JobHistoryEventHandler in handleEvent: EventType: AM_STARTED
> java.net.SocketTimeoutException: Read timed out
>       at java.net.SocketInputStream.socketRead0(Native Method)
>       at java.net.SocketInputStream.read(SocketInputStream.java:152)
>       at java.net.SocketInputStream.read(SocketInputStream.java:122)
>       at java.io.BufferedInputStream.fill(BufferedInputStream.java:235)
>       at java.io.BufferedInputStream.read1(BufferedInputStream.java:275)
>       at java.io.BufferedInputStream.read(BufferedInputStream.java:334)
>       at sun.net.www.http.HttpClient.parseHTTPHeader(HttpClient.java:687)
>       at sun.net.www.http.HttpClient.parseHTTP(HttpClient.java:633)
>       at 
> sun.net.www.protocol.http.HttpURLConnection.getInputStream(HttpURLConnection.java:1323)
>       at 
> java.net.HttpURLConnection.getResponseCode(HttpURLConnection.java:468)
>       at 
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:520)
>       at 
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.call(KMSClientProvider.java:505)
>       at 
> org.apache.hadoop.crypto.key.kms.KMSClientProvider.decryptEncryptedKey(KMSClientProvider.java:779)
>       at 
> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$3.call(LoadBalancingKMSClientProvider.java:185)
>       at 
> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider$3.call(LoadBalancingKMSClientProvider.java:181)
>       at 
> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.doOp(LoadBalancingKMSClientProvider.java:94)
>       at 
> org.apache.hadoop.crypto.key.kms.LoadBalancingKMSClientProvider.decryptEncryptedKey(LoadBalancingKMSClientProvider.java:181)
>       at 
> org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.decryptEncryptedKey(KeyProviderCryptoExtension.java:388)
>       at 
> org.apache.hadoop.hdfs.DFSClient.decryptEncryptedDataEncryptionKey(DFSClient.java:1420)
>       at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:1522)
>       at 
> org.apache.hadoop.hdfs.DFSClient.createWrappedOutputStream(DFSClient.java:1507)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:407)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem$6.doCall(DistributedFileSystem.java:400)
>       at 
> org.apache.hadoop.fs.FileSystemLinkResolver.resolve(FileSystemLinkResolver.java:81)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:400)
>       at 
> org.apache.hadoop.hdfs.DistributedFileSystem.create(DistributedFileSystem.java:343)
>       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:917)
>       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:898)
>       at org.apache.hadoop.fs.FileSystem.create(FileSystem.java:795)
>       at 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.createEventWriter(JobHistoryEventHandler.java:428)
>       at 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.setupEventWriter(JobHistoryEventHandler.java:468)
>       at 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler.handleEvent(JobHistoryEventHandler.java:553)
>       at 
> org.apache.hadoop.mapreduce.jobhistory.JobHistoryEventHandler$1.run(JobHistoryEventHandler.java:326)
>       at java.lang.Thread.run(Thread.java:745)
> {noformat}
> We should better handle this scenario and not throw an NPE.



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

---------------------------------------------------------------------
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