You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "Wei-Chiu Chuang (JIRA)" <ji...@apache.org> on 2018/07/02 21:23:00 UTC

[jira] [Commented] (HADOOP-15359) IPC client hang in kerberized cluster due to JDK deadlock

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

Wei-Chiu Chuang commented on HADOOP-15359:
------------------------------------------

I'm pretty sure I'm seeing the same bug, but this time it's more complex than what you reported, [~xiaochen]

I saw an Impalad (a HDFS client) blocked on talking to NameNodes, and there are several threads involved. Posting them here so others may find it some day.

This cluster is on CDH5.15, Kerberized, Hadoop at-rest encryption, Hadoop RPC encryption, Hadoop data transport encryption. Somehow, all threads were in blocked state, and here are the most relevant threads:

It started with this thread:
{noformat}
Thread 15180: (state = BLOCKED)
 - java.util.Hashtable.get(java.lang.Object) @bci=0, line=362 (Compiled frame)
 - java.util.Properties.getProperty(java.lang.String) @bci=2, line=969 (Compiled frame)
 - java.security.Provider.getProperty(java.lang.String) @bci=6, line=645 (Compiled frame)
 - java.security.Security.getProviderProperty(java.lang.String, java.security.Provider) @bci=2, line=264 (Compiled frame)
 - java.security.Security.isCriterionSatisfied(java.security.Provider, java.lang.String, java.lang.String, java.lang.String, java.lang.String) @bci=58, line=930 (Compiled frame)
 - java.security.Security.getProvidersNotUsingCache(java.lang.String, java.lang.String, java.lang.String, java.lang.String, java.security.Provider[]) @bci=30, line=905 (Compiled frame)
 - java.security.Security.getAllQualifyingCandidates(java.lang.String, java.lang.String, java.security.Provider[]) @bci=29, line=893 (Compiled frame)
 - java.security.Security.getProviders(java.util.Map) @bci=81, line=623 (Compiled frame)
 - java.security.Security.getProviders(java.lang.String) @bci=59, line=547 (Compiled frame)
 - javax.security.sasl.Sasl.createSaslClient(java.lang.String[], java.lang.String, java.lang.String, java.lang.String, java.util.Map, javax.security.auth.callback.CallbackHandler) @bci=68, line=374 (Compiled frame)
 - org.apache.hadoop.hbase.security.HBaseSaslRpcClient.createKerberosSaslClient(java.lang.String[], java.lang.String, java.lang.String) @bci=9, line=148 (Compiled frame)
 - org.apache.hadoop.hbase.security.HBaseSaslRpcClient.<init>(org.apache.hadoop.hbase.security.AuthMethod, org.apache.hadoop.security.token.Token, java.lang.String, boolean, java.lang.String) @bci=277, line=128 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupSaslConnection(java.io.InputStream, java.io.OutputStream) @bci=48, line=615 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.access$700(org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection, java.io.InputStream, java.io.OutputStream) @bci=3, line=163 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run() @bci=12, line=744 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection$2.run() @bci=1, line=741 (Compiled frame)
 - java.security.AccessController.doPrivileged(java.security.PrivilegedExceptionAction, java.security.AccessControlContext) @bci=0 (Compiled frame)
 - javax.security.auth.Subject.doAs(javax.security.auth.Subject, java.security.PrivilegedExceptionAction) @bci=42, line=422 (Compiled frame)
 - org.apache.hadoop.security.UserGroupInformation.doAs(java.security.PrivilegedExceptionAction) @bci=14, line=1920 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.setupIOstreams() @bci=305, line=741 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.writeRequest(org.apache.hadoop.hbase.ipc.Call, int, org.apache.htrace.Span) @bci=152, line=907 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl$Connection.tracedWriteRequest(org.apache.hadoop.hbase.ipc.Call, int, org.apache.htrace.Span) @bci=10, line=874 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.RpcClientImpl.call(org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController, com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.Message, com.google.protobuf.Message, org.apache.hadoop.hbase.security.User, java.net.InetSocketAddress, org.apache.hadoop.hbase.client.MetricsConnection$CallStats) @bci=146, line=1246 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.AbstractRpcClient.callBlockingMethod(com.google.protobuf.Descriptors$MethodDescriptor, org.apache.hadoop.hbase.ipc.PayloadCarryingRpcController, com.google.protobuf.Message, com.google.protobuf.Message, org.apache.hadoop.hbase.security.User, java.net.InetSocketAddress) @bci=37, line=227 (Compiled frame)
 - org.apache.hadoop.hbase.ipc.AbstractRpcClient$BlockingRpcChannelImplementation.callBlockingMethod(com.google.protobuf.Descriptors$MethodDescriptor, com.google.protobuf.RpcController, com.google.protobuf.Message, com.google.protobuf.Message) @bci=73, line=336 (Compiled frame)
 - org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ClientService$BlockingStub.scan(com.google.protobuf.RpcController, org.apache.hadoop.hbase.protobuf.generated.ClientProtos$ScanRequest) @bci=24, line=34094 (Compiled frame)
 - org.apache.hadoop.hbase.client.ScannerCallable.openScanner() @bci=30, line=400 (Compiled frame)
 - org.apache.hadoop.hbase.client.ScannerCallable.call(int) @bci=52, line=204 (Compiled frame)
 - org.apache.hadoop.hbase.client.ScannerCallable.call(int) @bci=2, line=65 (Compiled frame)
 - org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithoutRetries(org.apache.hadoop.hbase.client.RetryingCallable, int) @bci=16, line=210 (Compiled frame)
 - org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(int) @bci=18, line=397 (Compiled frame)
 - org.apache.hadoop.hbase.client.ScannerCallableWithReplicas$RetryingRPC.call(int) @bci=2, line=371 (Compiled frame)
 - org.apache.hadoop.hbase.client.RpcRetryingCaller.callWithRetries(org.apache.hadoop.hbase.client.RetryingCallable, int) @bci=64, line=136 (Compiled frame)
 - org.apache.hadoop.hbase.client.ResultBoundedCompletionService$QueueingFuture.run() @bci=20, line=80 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker) @bci=95, line=1142 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=617 (Compiled frame)
 - java.lang.Thread.run() @bci=11, line=745 (Compiled frame)
{noformat}

it blocked the following thread:

{noformat}
Thread 24817: (state = BLOCKED)
 - java.security.Provider.getService(java.lang.String, java.lang.String) @bci=0, line=1035 (Compiled frame)
 - sun.security.jca.ProviderList$ServiceList.tryGet(int) @bci=159, line=444 (Compiled frame)
 - sun.security.jca.ProviderList$ServiceList.access$200(sun.security.jca.ProviderList$ServiceList, int) @bci=2, line=376 (Compiled frame)
 - sun.security.jca.ProviderList$ServiceList$1.hasNext() @bci=8, line=486 (Compiled frame)
 - javax.crypto.Cipher.getInstance(java.lang.String) @bci=88, line=513 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.AesDkCrypto.getCipher(byte[], byte[], int) @bci=22, line=148 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.DkCrypto.dr(byte[], byte[]) @bci=4, line=484 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.DkCrypto.dk(byte[], byte[]) @bci=4, line=447 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.AesDkCrypto.decryptCTS(byte[], int, byte[], byte[], int, int, boolean) @bci=70, line=394 (Compiled frame)
 - sun.security.jgss.krb5.WrapToken_v2.getData(byte[], int) @bci=57, line=141 (Compiled frame)
 - sun.security.jgss.krb5.WrapToken_v2.getData() @bci=10, line=105 (Compiled frame)
 - sun.security.jgss.krb5.Krb5Context.unwrap(byte[], int, int, org.ietf.jgss.MessageProp) @bci=136, line=1058 (Compiled frame)
 - sun.security.jgss.GSSContextImpl.unwrap(byte[], int, int, org.ietf.jgss.MessageProp) @bci=16, line=403 (Compiled frame)
 - com.sun.security.sasl.gsskerb.GssKrb5Base.unwrap(byte[], int, int) @bci=57, line=77 (Compiled frame)
 - org.apache.hadoop.security.SaslRpcClient$WrappedInputStream.readNextRpcPacket() @bci=166, line=617 (Compiled frame)
 - org.apache.hadoop.security.SaslRpcClient$WrappedInputStream.read(byte[], int, int) @bci=25, line=583 (Compiled frame)
 - java.io.FilterInputStream.read(byte[], int, int) @bci=7, line=133 (Compiled frame)
 - org.apache.hadoop.ipc.Client$Connection$PingInputStream.read(byte[], int, int) @bci=7, line=553 (Compiled frame)
 - java.io.BufferedInputStream.fill() @bci=214, line=246 (Compiled frame)
 - java.io.BufferedInputStream.read() @bci=12, line=265 (Compiled frame)
 - java.io.DataInputStream.readInt() @bci=4, line=387 (Compiled frame)
 - org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse() @bci=19, line=1113 (Compiled frame)
 - org.apache.hadoop.ipc.Client$Connection.run() @bci=62, line=1006 (Compiled frame)
{noformat}

and which blocked this thread, because socket writer must wait for reader (the above thread), according to [JDK-4509080
(ch) Streams inhibit concurrent reading & writing|https://bugs.openjdk.java.net/browse/JDK-4509080]:

{noformat}
Thread 32333: (state = BLOCKED)
 - java.security.Provider.getService(java.lang.String, java.lang.String) @bci=0, line=1035 (Compiled frame)
 - sun.security.jca.ProviderList$ServiceList.tryGet(int) @bci=159, line=444 (Compiled frame)
 - sun.security.jca.ProviderList$ServiceList.access$200(sun.security.jca.ProviderList$ServiceList, int) @bci=2, line=376 (Compiled frame)
 - sun.security.jca.ProviderList$ServiceList$1.hasNext() @bci=8, line=486 (Compiled frame)
 - javax.crypto.Cipher.getInstance(java.lang.String) @bci=88, line=513 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.AesDkCrypto.getCipher(byte[], byte[], int) @bci=22, line=148 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.DkCrypto.dr(byte[], byte[]) @bci=4, line=484 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.DkCrypto.dk(byte[], byte[]) @bci=4, line=447 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.AesDkCrypto.encryptCTS(byte[], int, byte[], byte[], byte[], int, int, boolean) @bci=70, line=324 (Compiled frame)
 - sun.security.krb5.internal.crypto.dk.AesDkCrypto.encryptRaw(byte[], int, byte[], byte[], int, int) @bci=46, line=256 (Compiled frame)
 - sun.security.krb5.internal.crypto.Aes256.encryptRaw(byte[], int, byte[], byte[], int, int) @bci=11, line=70 (Compiled frame)
 - sun.security.jgss.krb5.CipherHelper.aes256Encrypt(byte[], byte[], byte[], int, int, int) @bci=63, line=1377 (Compiled frame)
 - sun.security.jgss.krb5.CipherHelper.encryptData(sun.security.jgss.krb5.WrapToken_v2, byte[], byte[], byte[], int, int, int) @bci=58, line=723 (Compiled frame)
 - sun.security.jgss.krb5.WrapToken_v2.<init>(sun.security.jgss.krb5.Krb5Context, org.ietf.jgss.MessageProp, byte[], int, int) @bci=131, line=200 (Compiled frame)
 - sun.security.jgss.krb5.Krb5Context.wrap(byte[], int, int, org.ietf.jgss.MessageProp) @bci=121, line=926 (Compiled frame)
 - sun.security.jgss.GSSContextImpl.wrap(byte[], int, int, org.ietf.jgss.MessageProp) @bci=16, line=385 (Compiled frame)
 - com.sun.security.sasl.gsskerb.GssKrb5Base.wrap(byte[], int, int) @bci=57, line=103 (Compiled frame)
 - org.apache.hadoop.security.SaslRpcClient$WrappedOutputStream.write(byte[], int, int) @bci=48, line=636 (Compiled frame)
 - java.io.BufferedOutputStream.flushBuffer() @bci=20, line=82 (Compiled frame)
 - java.io.BufferedOutputStream.flush() @bci=1, line=140 (Compiled frame)
 - java.io.DataOutputStream.flush() @bci=4, line=123 (Compiled frame)
 - org.apache.hadoop.ipc.Client$Connection$3.run() @bci=149, line=1072 (Compiled frame)
 - java.util.concurrent.Executors$RunnableAdapter.call() @bci=4, line=511 (Compiled frame)
 - java.util.concurrent.FutureTask.run() @bci=42, line=266 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor.runWorker(java.util.concurrent.ThreadPoolExecutor$Worker) @bci=95, line=1142 (Compiled frame)
 - java.util.concurrent.ThreadPoolExecutor$Worker.run() @bci=5, line=617 (Compiled frame)
 - java.lang.Thread.run() @bci=11, line=745 (Compiled frame)
{noformat}

{noformat}
Thread 17213: (state = BLOCKED)
 - sun.misc.Unsafe.park(boolean, long) @bci=0 (Compiled frame; information may be imprecise)
 - java.util.concurrent.locks.LockSupport.park(java.lang.Object) @bci=14, line=175 (Compiled frame)
 - java.util.concurrent.FutureTask.awaitDone(boolean, long) @bci=165, line=429 (Compiled frame)
 - java.util.concurrent.FutureTask.get() @bci=13, line=191 (Compiled frame)
 - org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(org.apache.hadoop.ipc.Client$Call) @bci=94, line=1088 (Compiled frame)
 - org.apache.hadoop.ipc.Client.call(org.apache.hadoop.ipc.RPC$RpcKind, org.apache.hadoop.io.Writable, org.apache.hadoop.ipc.Client$ConnectionId, int, java.util.concurrent.atomic.AtomicBoolean) @bci=25, line=1483 (Compiled frame)
 - org.apache.hadoop.ipc.Client.call(org.apache.hadoop.ipc.RPC$RpcKind, org.apache.hadoop.io.Writable, org.apache.hadoop.ipc.Client$ConnectionId, java.util.concurrent.atomic.AtomicBoolean) @bci=7, line=1441 (Compiled frame)
 - org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(java.lang.Object, java.lang.reflect.Method, java.lang.Object[]) @bci=260, line=230 (Compiled frame)
 - com.sun.proxy.$Proxy10.getBlockLocations(com.google.protobuf.RpcController, org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$GetBlockLocationsRequestProto) @bci=20 (Compiled frame)
 - org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(java.lang.String, long, long) @bci=28, line=268 (Compiled frame)
 - sun.reflect.GeneratedMethodAccessor8.invoke(java.lang.Object, java.lang.Object[]) @bci=246 (Compiled frame)
 - sun.reflect.DelegatingMethodAccessorImpl.invoke(java.lang.Object, java.lang.Object[]) @bci=6, line=43 (Compiled frame)
 - java.lang.reflect.Method.invoke(java.lang.Object, java.lang.Object[]) @bci=56, line=498 (Compiled frame)
 - org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(java.lang.reflect.Method, java.lang.Object[]) @bci=21, line=258 (Compiled frame)
 - org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(java.lang.Object, java.lang.reflect.Method, java.lang.Object[]) @bci=105, line=104 (Compiled frame)
 - com.sun.proxy.$Proxy11.getBlockLocations(java.lang.String, long, long) @bci=31 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(org.apache.hadoop.hdfs.protocol.ClientProtocol, java.lang.String, long, long) @bci=5, line=1324 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(java.lang.String, long, long) @bci=18, line=1311 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(java.lang.String, long) @bci=10, line=1299 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength() @bci=9, line=315 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSInputStream.openInfo() @bci=9, line=280 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSInputStream.<init>(org.apache.hadoop.hdfs.DFSClient, java.lang.String, boolean) @bci=147, line=267 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.open(java.lang.String, int, boolean) @bci=21, line=1630 (Compiled frame)
 - org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(org.apache.hadoop.fs.Path) @bci=26, line=339 (Compiled frame)
 - org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(org.apache.hadoop.fs.Path) @bci=2, line=335 (Compiled frame)
 - org.apache.hadoop.fs.FileSystemLinkResolver.resolve(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path) @bci=22, line=81 (Compiled frame)
 - org.apache.hadoop.hdfs.DistributedFileSystem.open(org.apache.hadoop.fs.Path, int) @bci=35, line=335 (Compiled frame)

{noformat}
and which blocked this thread:

{noformat}
Thread 17212: (state = BLOCKED)
 - org.apache.hadoop.ipc.Client$Connection.sendRpcRequest(org.apache.hadoop.ipc.Client$Call) @bci=67, line=1053 (Compiled frame)
 - org.apache.hadoop.ipc.Client.call(org.apache.hadoop.ipc.RPC$RpcKind, org.apache.hadoop.io.Writable, org.apache.hadoop.ipc.Client$ConnectionId, int, java.util.concurrent.atomic.AtomicBoolean) @bci=25, line=1483 (Compiled frame)
 - org.apache.hadoop.ipc.Client.call(org.apache.hadoop.ipc.RPC$RpcKind, org.apache.hadoop.io.Writable, org.apache.hadoop.ipc.Client$ConnectionId, java.util.concurrent.atomic.AtomicBoolean) @bci=7, line=1441 (Compiled frame)
 - org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(java.lang.Object, java.lang.reflect.Method, java.lang.Object[]) @bci=260, line=230 (Compiled frame)
 - com.sun.proxy.$Proxy10.getBlockLocations(com.google.protobuf.RpcController, org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos$GetBlockLocationsRequestProto) @bci=20 (Compiled frame)
 - org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.getBlockLocations(java.lang.String, long, long) @bci=28, line=268 (Compiled frame)
 - sun.reflect.GeneratedMethodAccessor8.invoke(java.lang.Object, java.lang.Object[]) @bci=246 (Compiled frame)
 - sun.reflect.DelegatingMethodAccessorImpl.invoke(java.lang.Object, java.lang.Object[]) @bci=6, line=43 (Compiled frame)
 - java.lang.reflect.Method.invoke(java.lang.Object, java.lang.Object[]) @bci=56, line=498 (Compiled frame)
 - org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(java.lang.reflect.Method, java.lang.Object[]) @bci=21, line=258 (Compiled frame)
 - org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(java.lang.Object, java.lang.reflect.Method, java.lang.Object[]) @bci=105, line=104 (Compiled frame)
 - com.sun.proxy.$Proxy11.getBlockLocations(java.lang.String, long, long) @bci=31 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.callGetBlockLocations(org.apache.hadoop.hdfs.protocol.ClientProtocol, java.lang.String, long, long) @bci=5, line=1324 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(java.lang.String, long, long) @bci=18, line=1311 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.getLocatedBlocks(java.lang.String, long) @bci=10, line=1299 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSInputStream.fetchLocatedBlocksAndGetLastBlockLength() @bci=9, line=315 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSInputStream.openInfo() @bci=9, line=280 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSInputStream.<init>(org.apache.hadoop.hdfs.DFSClient, java.lang.String, boolean) @bci=147, line=267 (Compiled frame)
 - org.apache.hadoop.hdfs.DFSClient.open(java.lang.String, int, boolean) @bci=21, line=1630 (Compiled frame)
 - org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(org.apache.hadoop.fs.Path) @bci=26, line=339 (Compiled frame)
 - org.apache.hadoop.hdfs.DistributedFileSystem$4.doCall(org.apache.hadoop.fs.Path) @bci=2, line=335 (Compiled frame)
 - org.apache.hadoop.fs.FileSystemLinkResolver.resolve(org.apache.hadoop.fs.FileSystem, org.apache.hadoop.fs.Path) @bci=22, line=81 (Compiled frame)
 - org.apache.hadoop.hdfs.DistributedFileSystem.open(org.apache.hadoop.fs.Path, int) @bci=35, line=335 (Compiled frame)

{noformat}

Incidentally, I suspect this is related to HADOOP-15530 and HADOOP-15538.

> IPC client hang in kerberized cluster due to JDK deadlock
> ---------------------------------------------------------
>
>                 Key: HADOOP-15359
>                 URL: https://issues.apache.org/jira/browse/HADOOP-15359
>             Project: Hadoop Common
>          Issue Type: Bug
>          Components: ipc
>    Affects Versions: 2.6.0, 2.8.0, 3.0.0
>            Reporter: Xiao Chen
>            Priority: Major
>         Attachments: 1.jstack, 2.jstack
>
>
> In a recent internal testing, we have found a DFS client hang. Further inspecting jstack shows the following:
> {noformat}
> "IPC Client (552936351) connection toHOSTNAME:8020 from PRINCIPAL" #7468 daemon prio=5 os_prio=0 tid=0x00007f6bb306c000 nid=0x1c76e waiting for monitor entry [0x00007f6bc2bd6000]
>    java.lang.Thread.State: BLOCKED (on object monitor)
>         at java.security.Provider.getService(Provider.java:1035)
>         - waiting to lock <0x0000000080277040> (a sun.security.provider.Sun)
>         at sun.security.jca.ProviderList$ServiceList.tryGet(ProviderList.java:444)
>         at sun.security.jca.ProviderList$ServiceList.access$200(ProviderList.java:376)
>         at sun.security.jca.ProviderList$ServiceList$1.hasNext(ProviderList.java:486)
>         at javax.crypto.Cipher.getInstance(Cipher.java:513)
>         at sun.security.krb5.internal.crypto.dk.Des3DkCrypto.getCipher(Des3DkCrypto.java:202)
>         at sun.security.krb5.internal.crypto.dk.DkCrypto.dr(DkCrypto.java:484)
>         at sun.security.krb5.internal.crypto.dk.DkCrypto.dk(DkCrypto.java:447)
>         at sun.security.krb5.internal.crypto.dk.DkCrypto.calculateChecksum(DkCrypto.java:413)
>         at sun.security.krb5.internal.crypto.Des3.calculateChecksum(Des3.java:59)
>         at sun.security.jgss.krb5.CipherHelper.calculateChecksum(CipherHelper.java:231)
>         at sun.security.jgss.krb5.MessageToken.getChecksum(MessageToken.java:466)
>         at sun.security.jgss.krb5.MessageToken.verifySignAndSeqNumber(MessageToken.java:374)
>         at sun.security.jgss.krb5.WrapToken.getDataFromBuffer(WrapToken.java:284)
>         at sun.security.jgss.krb5.WrapToken.getData(WrapToken.java:209)
>         at sun.security.jgss.krb5.WrapToken.getData(WrapToken.java:182)
>         at sun.security.jgss.krb5.Krb5Context.unwrap(Krb5Context.java:1053)
>         at sun.security.jgss.GSSContextImpl.unwrap(GSSContextImpl.java:403)
>         at com.sun.security.sasl.gsskerb.GssKrb5Base.unwrap(GssKrb5Base.java:77)
>         at org.apache.hadoop.security.SaslRpcClient$WrappedInputStream.readNextRpcPacket(SaslRpcClient.java:617)
>         at org.apache.hadoop.security.SaslRpcClient$WrappedInputStream.read(SaslRpcClient.java:583)
>         - locked <0x0000000083444878> (a java.nio.HeapByteBuffer)
>         at java.io.FilterInputStream.read(FilterInputStream.java:133)
>         at org.apache.hadoop.ipc.Client$Connection$PingInputStream.read(Client.java:553)
>         at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
>         at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
>         - locked <0x00000000834448c0> (a java.io.BufferedInputStream)
>         at java.io.DataInputStream.readInt(DataInputStream.java:387)
>         at org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1113)
>         at org.apache.hadoop.ipc.Client$Connection.run(Client.java:1006)
> {noformat}
> and at the end of jstack:
> {noformat}
> Found one Java-level deadlock:
> =============================
> "IPC Parameter Sending Thread #29":
>   waiting to lock monitor 0x0000000017ff49f8 (object 0x0000000080277040, a sun.security.provider.Sun),
>   which is held by UNKNOWN_owner_addr=0x0000000050607000
> Java stack information for the threads listed above:
> ===================================================
> "IPC Parameter Sending Thread #29":
>         at java.security.Provider.getService(Provider.java:1035)
>         - waiting to lock <0x0000000080277040> (a sun.security.provider.Sun)
>         at sun.security.jca.ProviderList$ServiceList.tryGet(ProviderList.java:437)
>         at sun.security.jca.ProviderList$ServiceList.access$200(ProviderList.java:376)
>         at sun.security.jca.ProviderList$ServiceList$1.hasNext(ProviderList.java:486)
>         at javax.crypto.SecretKeyFactory.nextSpi(SecretKeyFactory.java:293)
>         - locked <0x00000000834386b8> (a java.lang.Object)
>         at javax.crypto.SecretKeyFactory.<init>(SecretKeyFactory.java:121)
>         at javax.crypto.SecretKeyFactory.getInstance(SecretKeyFactory.java:160)
>         at sun.security.krb5.internal.crypto.dk.Des3DkCrypto.getCipher(Des3DkCrypto.java:187)
>         at sun.security.krb5.internal.crypto.dk.DkCrypto.dr(DkCrypto.java:484)
>         at sun.security.krb5.internal.crypto.dk.DkCrypto.dk(DkCrypto.java:447)
>         at sun.security.krb5.internal.crypto.dk.DkCrypto.calculateChecksum(DkCrypto.java:413)
>         at sun.security.krb5.internal.crypto.Des3.calculateChecksum(Des3.java:59)
>         at sun.security.jgss.krb5.CipherHelper.calculateChecksum(CipherHelper.java:231)
>         at sun.security.jgss.krb5.MessageToken.getChecksum(MessageToken.java:466)
>         at sun.security.jgss.krb5.MessageToken.genSignAndSeqNumber(MessageToken.java:315)
>         at sun.security.jgss.krb5.WrapToken.<init>(WrapToken.java:422)
>         at sun.security.jgss.krb5.Krb5Context.wrap(Krb5Context.java:922)
>         at sun.security.jgss.GSSContextImpl.wrap(GSSContextImpl.java:385)
>         at com.sun.security.sasl.gsskerb.GssKrb5Base.wrap(GssKrb5Base.java:103)
>         at org.apache.hadoop.security.SaslRpcClient$WrappedOutputStream.write(SaslRpcClient.java:636)
>         at java.io.BufferedOutputStream.flushBuffer(BufferedOutputStream.java:82)
>         at java.io.BufferedOutputStream.flush(BufferedOutputStream.java:140)
>         - locked <0x00000000834389a8> (a java.io.BufferedOutputStream)
>         at java.io.DataOutputStream.flush(DataOutputStream.java:123)
>         at org.apache.hadoop.ipc.Client$Connection$3.run(Client.java:1072)
>         - locked <0x00000000834389c0> (a java.io.DataOutputStream)
>         at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:511)
>         at java.util.concurrent.FutureTask.run(FutureTask.java:266)
>         at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
>         at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
>         at java.lang.Thread.run(Thread.java:748)
> {noformat}
> After some research, the closest I found is https://bugs.openjdk.java.net/browse/JDK-7092821 . 
> Filing this jira for discussions.
> JDK version used was 1.8.0_144.



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

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