Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 5481A200B5F for ; Fri, 29 Jul 2016 18:28:23 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 53196160A6E; Fri, 29 Jul 2016 16:28:23 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 70CC1160AA7 for ; Fri, 29 Jul 2016 18:28:22 +0200 (CEST) Received: (qmail 72026 invoked by uid 500); 29 Jul 2016 16:28:21 -0000 Mailing-List: contact mapreduce-issues-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list mapreduce-issues@hadoop.apache.org Received: (qmail 71767 invoked by uid 99); 29 Jul 2016 16:28:21 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 29 Jul 2016 16:28:21 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id B212A2C0D67 for ; Fri, 29 Jul 2016 16:28:20 +0000 (UTC) Date: Fri, 29 Jul 2016 16:28:20 +0000 (UTC) From: "Xiao Chen (JIRA)" To: mapreduce-issues@hadoop.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (MAPREDUCE-6654) Possible NPE in JobHistoryEventHandler#handleEvent MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Fri, 29 Jul 2016 16:28:23 -0000 [ https://issues.apache.org/jira/browse/MAPREDUCE-6654?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15399617#comment-15399617 ] Xiao Chen commented on MAPREDUCE-6654: -------------------------------------- Thanks [~djp] for working on this. bq. 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. Sounds reasonable - I think you meant to say so no NPE get throw. :) > 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 > > 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-unsubscribe@hadoop.apache.org For additional commands, e-mail: mapreduce-issues-help@hadoop.apache.org