[jira] [Comment Edited] (HDFS-6532) Intermittent test failure org.apache.hadoop.hdfs.TestCrcCorruption.testCorruptionDuringWrt
[ https://issues.apache.org/jira/browse/HDFS-6532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15550368#comment-15550368 ] Xiao Chen edited comment on HDFS-6532 at 10/6/16 6:05 AM: -- Thanks for the comment, [~linyiqun]. Took a jstack when right before the test timed out. It has: {noformat} "ResponseProcessor for block BP-341806944-172.17.0.1-1475696115790:blk_1073741825_1001" 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:79) at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87) 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.doIO(SocketIOWithTimeout.java:157) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118) at java.io.FilterInputStream.read(FilterInputStream.java:83) at java.io.FilterInputStream.read(FilterInputStream.java:83) at org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:2247) at org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck.readFields(PipelineAck.java:235) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSOutputStream.java:1017) "PacketResponder: BP-341806944-172.17.0.1-1475696115790:blk_1073741825_1001, type=HAS_DOWNSTREAM_IN_PIPELINE" 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:79) at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87) 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.doIO(SocketIOWithTimeout.java:157) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118) at java.io.FilterInputStream.read(FilterInputStream.java:83) at java.io.FilterInputStream.read(FilterInputStream.java:83) at org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:2247) at org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck.readFields(PipelineAck.java:235) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:1303) at java.lang.Thread.run(Thread.java:745) "DataXceiver for client DFSClient_NONMAPREDUCE_27347732_8 at /127.0.0.1:36783 [Receiving block BP-341806944-172.17.0.1-1475696115790:blk_1073741825_1001]" 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:79) at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:87) 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.doIO(SocketIOWithTimeout.java:157) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131) 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 java.io.DataInputStream.read(DataInputStream.java:149) at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:199) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(PacketReceiver.java:213) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:134) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:109) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:502) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:900) at
[jira] [Comment Edited] (HDFS-6532) Intermittent test failure org.apache.hadoop.hdfs.TestCrcCorruption.testCorruptionDuringWrt
[ https://issues.apache.org/jira/browse/HDFS-6532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15546919#comment-15546919 ] Xiao Chen edited comment on HDFS-6532 at 10/4/16 11:29 PM: --- Looked more into this. For failed cases, we see (copied from the 'select-timeout' attachment): {noformat} 2016-10-04 22:10:24,365 INFO hdfs.DFSOutputStream (DFSOutputStream.java:run(1114)) - == java.io.InterruptedIOException: Interruped while waiting for IO on channel java.nio.channels.SocketChannel[closed]. 28459 millis timeout left. at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:352) at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:157) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:118) at java.io.FilterInputStream.read(FilterInputStream.java:83) at java.io.FilterInputStream.read(FilterInputStream.java:83) at org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:2247) at org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck.readFields(PipelineAck.java:235) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSOutputStream.java:1015) {noformat} And for the success cases, we see: {noformat} 2016-10-04 15:13:15,271 INFO hdfs.DFSOutputStream (DFSOutputStream.java:run(1116)) - == java.io.IOException: Bad response ERROR for block BP-1283991366-172.16.3.181-1475619192335:blk_1073741825_1001 from datanode DatanodeInfoWithStorage[127.0.0.1:61321,DS-720243dd-55b6-49ef-ae55-4462e20260d5,DISK] at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSOutputStream.java:1053) {noformat} and {noformat} 2016-10-04 15:13:16,084 INFO hdfs.DFSOutputStream (DFSOutputStream.java:run(1116)) - == java.io.EOFException: Premature EOF: no length prefix available at org.apache.hadoop.hdfs.protocolPB.PBHelper.vintPrefixed(PBHelper.java:2249) at org.apache.hadoop.hdfs.protocol.datatransfer.PipelineAck.readFields(PipelineAck.java:235) at org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer$ResponseProcessor.run(DFSOutputStream.java:1017) {noformat} I printed the exception from [this line|https://github.com/apache/hadoop/blob/44f48ee96ee6b2a3909911c37bfddb0c963d5ffc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java#L1149]. So in the failed cases, the responder is running in [this loop|https://github.com/apache/hadoop/blob/44f48ee96ee6b2a3909911c37bfddb0c963d5ffc/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java#L708], until the following exception is thrown {noformat}2016-10-04 22:36:40,403 INFO datanode.DataNode (BlockReceiver.java:receiveBlock(941)) - Exception for BP-2046749708-172.17.0.1-1475620536833:blk_1073741826_1005 java.net.SocketTimeoutException: 6 millis timeout while waiting for channel to be ready for read. ch : java.nio.channels.SocketChannel[connected local=/127.0.0.1:42956 remote=/127.0.0.1:56324] at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:164) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:161) at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:131) 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 java.io.DataInputStream.read(DataInputStream.java:149) at org.apache.hadoop.io.IOUtils.readFully(IOUtils.java:199) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doReadFully(PacketReceiver.java:213) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.doRead(PacketReceiver.java:134) at org.apache.hadoop.hdfs.protocol.datatransfer.PacketReceiver.receiveNextPacket(PacketReceiver.java:109) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:502) at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:900) at org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:802) at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:169) at org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:106) at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:246) at java.lang.Thread.run(Thread.java:745) 2016-10-04 22:36:40,469 INFO
[jira] [Comment Edited] (HDFS-6532) Intermittent test failure org.apache.hadoop.hdfs.TestCrcCorruption.testCorruptionDuringWrt
[ https://issues.apache.org/jira/browse/HDFS-6532?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15448770#comment-15448770 ] Yiqun Lin edited comment on HDFS-6532 at 8/30/16 11:24 AM: --- I looked into this issue again and I might find the root cause. As [~kihwal] had mentioned, the failed case will not print the following infos {code} (TestCrcCorruption.java:testCorruptionDuringWrt(140)) - Got expected exception java.io.IOException: Failing write. Tried pipeline recovery 5 times without success. {code} Instead of that, the failed case will print infos like these: {code} (TestCrcCorruption.java:testCorruptionDuringWrt(140)) - Got expected exception java.io.InterruptedIOException: Interrupted while waiting for data to be acknowledged by pipeline at org.apache.hadoop.hdfs.DataStreamer.waitForAckedSeqno(DataStreamer.java:775) at org.apache.hadoop.hdfs.DFSOutputStream.flushInternal(DFSOutputStream.java:697) at org.apache.hadoop.hdfs.DFSOutputStream.closeImpl(DFSOutputStream.java:778) at org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:755) at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72) at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101) {code} That means the program has returned before do the recover pipeline operations sometimes. The related codes: {code:title=DataStreamer.java|borderStyle=solid} private boolean processDatanodeOrExternalError() throws IOException { if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) { return false; } LOG.debug("start process datanode/external error, {}", this); // If the response has not closed, this method will just return if (response != null) { LOG.info("Error Recovery for " + block + " waiting for responder to exit. "); return true; } closeStream(); ... {code} I looked into the code and I thought there was a bug to cause that, the related codes: {code:title=DataStreamer.java|borderStyle=solid} public void run() { long lastPacket = Time.monotonicNow(); TraceScope scope = null; while (!streamerClosed && dfsClient.clientRunning) { // if the Responder encountered an error, shutdown Responder if (errorState.hasError() && response != null) { try { response.close(); response.join(); response = null; } catch (InterruptedException e) { // If interruptedException happens, the response will not be set to null LOG.warn("Caught exception", e); } } // Here need add a finally block to set response as null ... {code} I think we should move the line {{response = null;}} into {{finally}} block. Finally attach a patch for this. This test has failed intermitly for a long time, hope my patch can make sense. Softly ping [~xiaochen], [~kihwal] and [~yzhangal] for the comments. Thanks. was (Author: linyiqun): I looked into this issue again and I might find the root cause. As [~kihwal] had mentioned, the failed case will not print the following infos {code} (TestCrcCorruption.java:testCorruptionDuringWrt(140)) - Got expected exception java.io.IOException: Failing write. Tried pipeline recovery 5 times without success. {code} That means the program has returned before do the recover pipeline operations sometimes. The related codes: {code:title=DataStreamer.java|borderStyle=solid} private boolean processDatanodeOrExternalError() throws IOException { if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) { return false; } LOG.debug("start process datanode/external error, {}", this); // If the response has not closed, this method will just return if (response != null) { LOG.info("Error Recovery for " + block + " waiting for responder to exit. "); return true; } closeStream(); ... {code} I looked into the code and I thought there was a bug to cause that, the related codes: {code:title=DataStreamer.java|borderStyle=solid} public void run() { long lastPacket = Time.monotonicNow(); TraceScope scope = null; while (!streamerClosed && dfsClient.clientRunning) { // if the Responder encountered an error, shutdown Responder if (errorState.hasError() && response != null) { try { response.close(); response.join(); response = null; } catch (InterruptedException e) { // If interruptedException happens, the response will not be set to null LOG.warn("Caught exception", e); } } // Here need add a finally block to set response as null ... {code} I think we should move the line {{response = null;}} into {{finally}} block. Finally attach a patch for this. This test has failed intermitly