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

Steve Loughran commented on HDFS-5755:
--------------------------------------

h1. Analysis

Main thread is blocked while DFS client is trying to connect
{code}
"main" prio=5 tid=0x00007feed2019800 nid=0x1903 runnable [0x0000000111152000]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.KQueueArrayWrapper.kevent0(Native Method)
        at sun.nio.ch.KQueueArrayWrapper.poll(KQueueArrayWrapper.java:200)
        at sun.nio.ch.KQueueSelectorImpl.doSelect(KQueueSelectorImpl.java:103)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87)
        - locked <0x00000007fb6aeee8> (a sun.nio.ch.Util$2)
        - locked <0x00000007fb6aee68> (a java.util.Collections$UnmodifiableSet)
        - locked <0x00000007fb6aece0> (a sun.nio.ch.KQueueSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:98)
        at 
org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:335)
        at 
org.apache.hadoop.net.SocketIOWithTimeout.connect(SocketIOWithTimeout.java:203)
        at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:529)
        at org.apache.hadoop.net.NetUtils.connect(NetUtils.java:493)
        at 
org.apache.hadoop.ipc.Client$Connection.setupConnection(Client.java:547)
        - locked <0x00000007fb5ceea8> (a 
org.apache.hadoop.ipc.Client$Connection)
        at 
org.apache.hadoop.ipc.Client$Connection.setupIOstreams(Client.java:642)
        - locked <0x00000007fb5ceea8> (a 
org.apache.hadoop.ipc.Client$Connection)
        at org.apache.hadoop.ipc.Client$Connection.access$2600(Client.java:314)
        at org.apache.hadoop.ipc.Client.getConnection(Client.java:1399)
        at org.apache.hadoop.ipc.Client.call(Client.java:1318)
        at org.apache.hadoop.ipc.Client.call(Client.java:1300)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:206)
        at com.sun.proxy.$Proxy13.getApplications(Unknown Source)
{code}

SIGINT handler has called System.exit, which is running shutdown hooks

{code}
"SIGINT handler" daemon prio=5 tid=0x00007feed2403800 nid=0x3d13 in 
Object.wait() [0x0000000117a14000]
   java.lang.Thread.State: WAITING (on object monitor)
        at java.lang.Object.wait(Native Method)
        - waiting on <0x00000007fed58110> (a 
org.apache.hadoop.util.ShutdownHookManager$1)
        at java.lang.Thread.join(Thread.java:1280)
        - locked <0x00000007fed58110> (a 
org.apache.hadoop.util.ShutdownHookManager$1)
        at java.lang.Thread.join(Thread.java:1354)
        at 
java.lang.ApplicationShutdownHooks.runHooks(ApplicationShutdownHooks.java:106)
        at 
java.lang.ApplicationShutdownHooks$1.run(ApplicationShutdownHooks.java:46)
        at java.lang.Shutdown.runHooks(Shutdown.java:123)
        at java.lang.Shutdown.sequence(Shutdown.java:167)
        at java.lang.Shutdown.exit(Shutdown.java:212)
        - locked <0x00000007fed4a298> (a java.lang.Class for java.lang.Shutdown)
        at java.lang.Runtime.exit(Runtime.java:109)
        at java.lang.System.exit(System.java:962)
        at org.apache.hadoop.util.ExitUtil.terminate(ExitUtil.java:133)
        at org.apache.hadoop.util.ExitUtil.terminate(ExitUtil.java:192)
        at 
org.apache.hadoop.yarn.service.launcher.ServiceLauncher.exit(ServiceLauncher.java:270)
        at 
org.apache.hadoop.yarn.service.launcher.ServiceLauncher.interrupted(ServiceLauncher.java:257)
        at 
org.apache.hadoop.yarn.service.launcher.IrqHandler.handle(IrqHandler.java:70)
        at sun.misc.Signal$1.run(Signal.java:212)
        at java.lang.Thread.run(Thread.java:744)
{code}


And the shutdown hook blocks
{code}


{code}
"Thread-0" prio=5 tid=0x00007feed32ed800 nid=0x5707 waiting on condition 
[0x000000011a45d000]
   java.lang.Thread.State: TIMED_WAITING (sleeping)
        at java.lang.Thread.sleep(Native Method)
        at org.apache.hadoop.ipc.Client.stop(Client.java:1173)
        at org.apache.hadoop.ipc.ClientCache.stopClient(ClientCache.java:100)
        at 
org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.close(ProtobufRpcEngine.java:251)
        at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:626)
        at 
org.apache.hadoop.io.retry.DefaultFailoverProxyProvider.close(DefaultFailoverProxyProvider.java:57)
        at 
org.apache.hadoop.io.retry.RetryInvocationHandler.close(RetryInvocationHandler.java:206)
        at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:626)
        at 
org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.close(ClientNamenodeProtocolTranslatorPB.java:174)
        at org.apache.hadoop.ipc.RPC.stopProxy(RPC.java:621)
        at 
org.apache.hadoop.hdfs.DFSClient.closeConnectionToNamenode(DFSClient.java:738)
        at org.apache.hadoop.hdfs.DFSClient.close(DFSClient.java:794)
        - locked <0x00000007fec26860> (a org.apache.hadoop.hdfs.DFSClient)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.close(DistributedFileSystem.java:847)
        at org.apache.hadoop.fs.FileSystem$Cache.closeAll(FileSystem.java:2524)
        - locked <0x00000007fec254e0> (a org.apache.hadoop.fs.FileSystem$Cache)
        at 
org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer.run(FileSystem.java:2541)
        - locked <0x00000007fec254f8> (a 
org.apache.hadoop.fs.FileSystem$Cache$ClientFinalizer)
        at 
org.apache.hadoop.util.ShutdownHookManager$1.run(ShutdownHookManager.java:54)

{code}

h3. shutdown hook sequence:
# Hadoop shutdown hook triggers {{FileSystem.cache.closeAll()}}
# which invokes {{DFSClient.closeConnectionToNamenode()}}
# which tries to stop the RPC proxy
# which trickles all the way through to the IPC {{ClientCache.stopClient()}}
# which waits until all the client connections are closed
{code}
    // wait until all connections are closed
    while (!connections.isEmpty()) {
      try {
        Thread.sleep(100);
      } catch (InterruptedException e) {
      }
    }
{code}

As a result: no shutdown until all client connections are closed -hence no 
shutdown until all spinning/retrying client connections have succeeded.


> Filesystem close blocks SystemExit if a DFS client is trying to talk to a 
> nonexistent NN
> ----------------------------------------------------------------------------------------
>
>                 Key: HDFS-5755
>                 URL: https://issues.apache.org/jira/browse/HDFS-5755
>             Project: Hadoop HDFS
>          Issue Type: Bug
>    Affects Versions: 2.2.0
>            Reporter: Steve Loughran
>            Priority: Minor
>
> If a DFS client instance is spinning in a connection refused cycle, and you 
> try to stop the process via System.exit(), the exit call does not complete 
> until the client has actually given up. 
> That is: you can't exit a process with a standard kill while a DFS client 
> operation is blocking. 



--
This message was sent by Atlassian JIRA
(v6.1.5#6160)

Reply via email to