[ 
https://issues.apache.org/jira/browse/HDFS-15957?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated HDFS-15957:
----------------------------------
    Labels: pull-request-available  (was: )

> The ignored IOException in the RPC response sent by FSEditLogAsync can cause 
> the HDFS client to hang
> ----------------------------------------------------------------------------------------------------
>
>                 Key: HDFS-15957
>                 URL: https://issues.apache.org/jira/browse/HDFS-15957
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: fs async, namenode
>    Affects Versions: 3.2.2
>            Reporter: Haoze Wu
>            Priority: Critical
>              Labels: pull-request-available
>         Attachments: fsshell.txt, namenode.txt, reproduce.patch, 
> secondnamenode.txt
>
>          Time Spent: 10m
>  Remaining Estimate: 0h
>
>     In FSEditLogAsync, the RpcEdit notification in line 248 could be skipped, 
> because the possible exception (e.g., IOException) thrown in line 365 is 
> always ignored.
>  
> {code:java}
> //hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogAsync.java
> class FSEditLogAsync extends FSEditLog implements Runnable {
>   // ...
>   @Override
>   public void run() {
>     try {
>       while (true) {
>         boolean doSync;
>         Edit edit = dequeueEdit();
>         if (edit != null) {
>           // sync if requested by edit log.
>           doSync = edit.logEdit();
>           syncWaitQ.add(edit);
>         } else {
>           // sync when editq runs dry, but have edits pending a sync.
>           doSync = !syncWaitQ.isEmpty();
>         }
>         if (doSync) {
>           // normally edit log exceptions cause the NN to terminate, but tests
>           // relying on ExitUtil.terminate need to see the exception.
>           RuntimeException syncEx = null;
>           try {
>             logSync(getLastWrittenTxId());
>           } catch (RuntimeException ex) {
>             syncEx = ex;
>           }
>           while ((edit = syncWaitQ.poll()) != null) {
>             edit.logSyncNotify(syncEx);                               // line 
> 248
>           }
>         }
>       }
>     } catch (InterruptedException ie) {
>       LOG.info(Thread.currentThread().getName() + " was interrupted, 
> exiting");
>     } catch (Throwable t) {
>       terminate(t);
>     }
>   }
>   // the calling rpc thread will return immediately from logSync but the
>   // rpc response will not be sent until the edit is durable.
>   private static class RpcEdit extends Edit {
>     // ...
>     @Override
>     public void logSyncNotify(RuntimeException syncEx) {
>       try {
>         if (syncEx == null) {
>           call.sendResponse();                                        // line 
> 365
>         } else {
>           call.abortResponse(syncEx);
>         }
>       } catch (Exception e) {} // don't care if not sent.
>     }
>   }
> }
> {code}
>     The `call.sendResponse()` may throw an IOException. According to the 
> comment (“don’t care if not sent”) there, this exception is neither handled 
> nor printed in log. However, we suspect that some RPC responses sent there 
> may be critical, and there should be some retry mechanism.
>     We try to introduce a single IOException in line 365, and find that the 
> HDFS client (e.g., `bin/hdfs dfs -copyFromLocal ./foo.txt /1.txt`) may get 
> stuck forever (hang for >30min without any log). We can reproduce this 
> symptom in multiple ways. One of the simplest ways of reproduction is shown 
> as follows:
>  # Start a new empty HDFS cluster (1 namenode, 2 datanodes) with the default 
> configuration.
>  # Generate a file of 15MB for testing, by `fallocate -l 15000000 foo.txt`.
>  # Run the HDFS client `bin/hdfs dfs -copyFromLocal ./foo.txt /1.txt`.
>  # When line 365 is invoked the third time (it is invoked 6 times in total in 
> this experiment), inject an IOException there. (A patch for injecting the 
> exception this way is attached to reproduce the issue)
>     Then the client hangs forever, without any log. If we run `bin/hdfs dfs 
> -ls /` to check the file status, we can not see the expected 15MB `/1.txt` 
> file.
>     The jstack of the HDFS client shows that there is an RPC call infinitely 
> waiting.
> {code:java}
> "Thread-6" #18 daemon prio=5 os_prio=0 tid=0x00007f9cd5295800 nid=0x26b9 in 
> Object.wait() [0x00007f9ca354f000]
>    java.lang.Thread.State: WAITING (on object monitor)
>     at java.lang.Object.wait(Native Method)
>     - waiting on <0x000000071e709610> (a org.apache.hadoop.ipc.Client$Call)
>     at java.lang.Object.wait(Object.java:502)
>     at org.apache.hadoop.util.concurrent.AsyncGet$Util.wait(AsyncGet.java:59)
>     at org.apache.hadoop.ipc.Client.getRpcResponse(Client.java:1556)
>     - locked <0x000000071e709610> (a org.apache.hadoop.ipc.Client$Call)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1513)
>     at org.apache.hadoop.ipc.Client.call(Client.java:1410)
>     at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:233)
>     at 
> org.apache.hadoop.ipc.ProtobufRpcEngine$Invoker.invoke(ProtobufRpcEngine.java:118)
>     at com.sun.proxy.$Proxy9.addBlock(Unknown Source)
>     at 
> org.apache.hadoop.hdfs.protocolPB.ClientNamenodeProtocolTranslatorPB.addBlock(ClientNamenodeProtocolTranslatorPB.java:520)
>     at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)
>     at 
> sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)
>     at 
> sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)
>     at java.lang.reflect.Method.invoke(Method.java:498)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invokeMethod(RetryInvocationHandler.java:422)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeMethod(RetryInvocationHandler.java:165)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invoke(RetryInvocationHandler.java:157)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call.invokeOnce(RetryInvocationHandler.java:95)
>     - locked <0x000000071e681d28> (a 
> org.apache.hadoop.io.retry.RetryInvocationHandler$Call)
>     at 
> org.apache.hadoop.io.retry.RetryInvocationHandler.invoke(RetryInvocationHandler.java:359)
>     at com.sun.proxy.$Proxy10.addBlock(Unknown Source)
>     at 
> org.apache.hadoop.hdfs.DFSOutputStream.addBlock(DFSOutputStream.java:1081)
>     at 
> org.apache.hadoop.hdfs.DataStreamer.locateFollowingBlock(DataStreamer.java:1869)
>     at 
> org.apache.hadoop.hdfs.DataStreamer.nextBlockOutputStream(DataStreamer.java:1671)
>     at org.apache.hadoop.hdfs.DataStreamer.run(DataStreamer.java:715)"IPC 
> Client (440737101) connection to /127.0.0.1:9000 from whz" #16 daemon prio=5 
> os_prio=0 tid=0x00007f9cd5251800 nid=0x26b3 runnable [0x00007f9ca3752000]
>    java.lang.Thread.State: RUNNABLE
>     at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
>     at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:269)
>     at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:93)
>     at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:86)
>     - locked <0x000000071d243510> (a sun.nio.ch.Util$3)
>     - locked <0x000000071d243488> (a java.util.Collections$UnmodifiableSet)
>     - locked <0x000000071d2430c0> (a sun.nio.ch.EPollSelectorImpl)
>     at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:97)
>     at 
> org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:339)
>     at 
> org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:158)
>     at 
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161)
>     at 
> org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131)
>     at java.io.FilterInputStream.read(FilterInputStream.java:133)
>     at java.io.BufferedInputStream.fill(BufferedInputStream.java:246)
>     at java.io.BufferedInputStream.read(BufferedInputStream.java:265)
>     - locked <0x000000071d271828> (a java.io.BufferedInputStream)
>     at java.io.FilterInputStream.read(FilterInputStream.java:83)
>     at java.io.FilterInputStream.read(FilterInputStream.java:83)
>     at 
> org.apache.hadoop.ipc.Client$Connection$PingInputStream.read(Client.java:564)
>     at java.io.DataInputStream.readInt(DataInputStream.java:387)
>     at org.apache.hadoop.ipc.Client$IpcStreams.readResponse(Client.java:1886)
>     at 
> org.apache.hadoop.ipc.Client$Connection.receiveRpcResponse(Client.java:1195)
>     at org.apache.hadoop.ipc.Client$Connection.run(Client.java:1090)"main" #1 
> prio=5 os_prio=0 tid=0x00007f9cd4019800 nid=0x269a in Object.wait() 
> [0x00007f9cdce5a000]
>    java.lang.Thread.State: WAITING (on object monitor)
>     at java.lang.Object.wait(Native Method)
>     - waiting on <0x000000071e179400> (a java.util.LinkedList)
>     at java.lang.Object.wait(Object.java:502)
>     at 
> org.apache.hadoop.hdfs.DataStreamer.waitAndQueuePacket(DataStreamer.java:930)
>     - locked <0x000000071e179400> (a java.util.LinkedList)
>     at 
> org.apache.hadoop.hdfs.DFSOutputStream.enqueueCurrentPacket(DFSOutputStream.java:483)
>     at 
> org.apache.hadoop.hdfs.DFSOutputStream.enqueueCurrentPacketFull(DFSOutputStream.java:492)
>     - locked <0x000000071e01fb48> (a org.apache.hadoop.hdfs.DFSOutputStream)
>     at 
> org.apache.hadoop.hdfs.DFSOutputStream.writeChunk(DFSOutputStream.java:433)
>     - locked <0x000000071e01fb48> (a org.apache.hadoop.hdfs.DFSOutputStream)
>     at 
> org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunks(FSOutputSummer.java:217)
>     at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:125)
>     at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:111)
>     - locked <0x000000071e01fb48> (a org.apache.hadoop.hdfs.DFSOutputStream)
>     at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:57)
>     at java.io.DataOutputStream.write(DataOutputStream.java:107)
>     - locked <0x000000071e267270> (a 
> org.apache.hadoop.hdfs.client.HdfsDataOutputStream)
>     at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:96)
>     at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:68)
>     at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:129)
>     at 
> org.apache.hadoop.fs.shell.CommandWithDestination$TargetFileSystem.writeStreamToFile(CommandWithDestination.java:485)
>     at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyStreamToTarget(CommandWithDestination.java:407)
>     at 
> org.apache.hadoop.fs.shell.CommandWithDestination.copyFileToTarget(CommandWithDestination.java:342)
>     at 
> org.apache.hadoop.fs.shell.CopyCommands$Put.copyFile(CopyCommands.java:335)
>     at 
> org.apache.hadoop.fs.shell.CopyCommands$Put.copyFileToTarget(CopyCommands.java:343)
>     at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:277)
>     at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPath(CommandWithDestination.java:262)
>     at 
> org.apache.hadoop.fs.shell.Command.processPathInternal(Command.java:367)
>     at org.apache.hadoop.fs.shell.Command.processPaths(Command.java:331)
>     at 
> org.apache.hadoop.fs.shell.Command.processPathArgument(Command.java:304)
>     at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processPathArgument(CommandWithDestination.java:257)
>     at org.apache.hadoop.fs.shell.Command.processArgument(Command.java:286)
>     at org.apache.hadoop.fs.shell.Command.processArguments(Command.java:270)
>     at 
> org.apache.hadoop.fs.shell.CommandWithDestination.processArguments(CommandWithDestination.java:228)
>     at 
> org.apache.hadoop.fs.shell.CopyCommands$Put.processArguments(CopyCommands.java:303)
>     at 
> org.apache.hadoop.fs.shell.FsCommand.processRawArguments(FsCommand.java:120)
>     at org.apache.hadoop.fs.shell.Command.run(Command.java:177)
>     at org.apache.hadoop.fs.FsShell.run(FsShell.java:327)
>     at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:76)
>     at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:90)
>     at org.apache.hadoop.fs.FsShell.main(FsShell.java:390)
> {code}
>     The jstack for the namenode shows:
> {code:java}
> "FSEditLogAsync" #44 prio=5 os_prio=0 tid=0x00007f1af5802800 nid=0x45f0 
> waiting on condition [0x00007f1ac3090000]
>    java.lang.Thread.State: WAITING (parking)
>   at sun.misc.Unsafe.park(Native Method)
>   - parking to wait for  <0x00000005d6b1e198> (a 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject)
>   at java.util.concurrent.locks.LockSupport.park(LockSupport.java:175)
>   at 
> java.util.concurrent.locks.AbstractQueuedSynchronizer$ConditionObject.await(AbstractQueuedSynchronizer.java:2039)
>   at java.util.concurrent.ArrayBlockingQueue.take(ArrayBlockingQueue.java:403)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.dequeueEdit(FSEditLogAsync.java:221)
>   at 
> org.apache.hadoop.hdfs.server.namenode.FSEditLogAsync.run(FSEditLogAsync.java:230)
>   at java.lang.Thread.run(Thread.java:748)
> {code}
>     The full jstack files for the client, namenode and secondary namenode are 
> attached.We are not sure about the exact causal chain for the client hanging.
>     However, we find that adding a retry logic in the response sending will 
> fix the hanging issue. We will raise a pull request based on this retry logic.
>     We also attach the jstack of fsshell, namenode, and secondarynamenode for 
> reference.
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org

Reply via email to