[ https://issues.apache.org/jira/browse/HDFS-915?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12803970#action_12803970 ]
Todd Lipcon commented on HDFS-915: ---------------------------------- This still happens on trunk, and actually a little worse: [t...@monster01 hadoop-combined]$ top -d 0.1 -b | ./bin/hadoop fs -put - top-foo-9 ======> I issue kill -STOP. Around a minute later (read timeout time) I get, as expected: 10/01/22 16:23:31 WARN hdfs.DFSClient: DFSOutputStream ResponseProcessor exception for block blk_-5990414986184221392_1107java.io.IOException: Bad response ERROR for block blk_-5990414986184221392_1107 from datanode 192.168.42.41:42060 at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSClient.java:3071) ======> I hit C-\ to get a thread dump (elided the JVM threads which arent important) 2010-01-22 16:24:31 Full thread dump Java HotSpot(TM) 64-Bit Server VM (14.0-b16 mixed mode): "LeaseChecker" daemon prio=10 tid=0x00000000578eb800 nid=0x6fbe waiting on condition [0x0000000042c02000] java.lang.Thread.State: TIMED_WAITING (sleeping) at java.lang.Thread.sleep(Native Method) at org.apache.hadoop.hdfs.DFSClient$LeaseChecker.run(DFSClient.java:1289) at java.lang.Thread.run(Thread.java:619) "DataStreamer for file /user/todd/top-foo-9 block blk_-5990414986184221392_1107" daemon prio=10 tid=0x00000000578eb000 nid=0x6fbd runnable [0x0000000040803000] java.lang.Thread.State: RUNNABLE at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method) at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:215) at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65) at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69) - locked <0x00002aaadd18dc30> (a sun.nio.ch.Util$1) - locked <0x00002aaadd18dc18> (a java.util.Collections$UnmodifiableSet) - locked <0x00002aaadd18d8b8> (a sun.nio.ch.EPollSelectorImpl) at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80) at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:332) at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157) at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:146) at org.apache.hadoop.net.SocketOutputStream.write(SocketOutputStream.java:107) at java.io.BufferedOutputStream.write(BufferedOutputStream.java:105) - locked <0x00002aaaddaf9318> (a java.io.BufferedOutputStream) at java.io.DataOutputStream.write(DataOutputStream.java:90) - locked <0x00002aaaddaf92e8> (a java.io.DataOutputStream) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream$DataStreamer.run(DFSClient.java:2924) "main" prio=10 tid=0x00000000574e0800 nid=0x6f9a in Object.wait() [0x0000000040209000] java.lang.Thread.State: WAITING (on object monitor) at java.lang.Object.wait(Native Method) - waiting on <0x00002aaab34b0dd0> (a java.util.LinkedList) at java.lang.Object.wait(Object.java:485) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.waitAndQueuePacket(DFSClient.java:3561) - locked <0x00002aaab34b0dd0> (a java.util.LinkedList) at org.apache.hadoop.hdfs.DFSClient$DFSOutputStream.writeChunk(DFSClient.java:3620) - locked <0x00002aaaddaf97a0> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream) at org.apache.hadoop.fs.FSOutputSummer.writeChecksumChunk(FSOutputSummer.java:150) at org.apache.hadoop.fs.FSOutputSummer.write1(FSOutputSummer.java:100) at org.apache.hadoop.fs.FSOutputSummer.write(FSOutputSummer.java:86) - locked <0x00002aaaddaf97a0> (a org.apache.hadoop.hdfs.DFSClient$DFSOutputStream) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.write(FSDataOutputStream.java:49) at java.io.DataOutputStream.write(DataOutputStream.java:90) - locked <0x00002aaaddb32eb8> (a org.apache.hadoop.fs.FSDataOutputStream) at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:68) at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:45) at org.apache.hadoop.io.IOUtils.copyBytes(IOUtils.java:97) at org.apache.hadoop.fs.FsShell.copyFromStdin(FsShell.java:101) at org.apache.hadoop.fs.FsShell.copyFromLocal(FsShell.java:127) at org.apache.hadoop.fs.FsShell.run(FsShell.java:1846) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:65) at org.apache.hadoop.util.ToolRunner.run(ToolRunner.java:79) at org.apache.hadoop.fs.FsShell.main(FsShell.java:1977) ======> 6 minutes later, the TCP connection itself times out, and error recovery kicks in 10/01/22 16:30:33 WARN hdfs.DFSClient: Error Recovery for block blk_-5990414986184221392_1107 in pipeline 192.168.42.40:47076, 192.168.42.41:42060, 192.168.42.42:34557: bad datanode 192.168.42.41:42060 ======> and fails in some bizarre way - the writer is still writing into fs -put, but it dumps me back to my prompt [t...@monster01 hadoop-combined]$ > Hung DN stalls write pipeline for far longer than its timeout > ------------------------------------------------------------- > > Key: HDFS-915 > URL: https://issues.apache.org/jira/browse/HDFS-915 > Project: Hadoop HDFS > Issue Type: Bug > Components: hdfs client > Affects Versions: 0.20.1 > Reporter: Todd Lipcon > > After running kill -STOP on the datanode in the middle of a write pipeline, > the client takes far longer to recover than it should. The ResponseProcessor > times out in the correct interval, but doesn't interrupt the DataStreamer, > which appears to not be subject to the same timeout. The client only recovers > once the OS actually declares the TCP stream dead, which can take a very long > time. > I've experienced this on 0.20.1, haven't tried it yet on trunk or 0.21. -- This message is automatically generated by JIRA. - You can reply to this email to add a comment to the issue online.