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

Vinod Kumar Vavilapalli commented on MAPREDUCE-6654:
----------------------------------------------------

bq. To be clear, the events are not lost - it get tracked down with proper log 
messages which is exactly the same as writing other events (with event writer 
setup successfully before) when NN cannot be connected. The bottom line here is 
all event failures should be tracked with error log and get isolated properly 
so won't affect other following up events (and won't cause AM failed).
[~djp] / [~vvasudev], I am not sure if we are getting this right. We depend on 
reliable persistence of these events both in the UI as well as during Job 
recovery after AM restarts.

IIUC, before this patch, the job fails because it couldn't persist the 
information to the history. I think we are better off keeping the events in the 
queue in the same order and keep retrying till we can reconnect back to the 
FileSystem. Which reminds me, why isn't the DFSClient not looping till it 
connects back to the FileSystem?

> 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: Critical
>         Attachments: MAPREDUCE-6654-v2.1.patch, MAPREDUCE-6654-v2.patch, 
> MAPREDUCE-6654.patch
>
>
> 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