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

Xiao Chen commented on HDFS-6532:
---------------------------------

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 
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)
{noformat}
Attaching the full jstack for information.

Hi [~kihwal],
do you have any insight to this? IIUC, DFSOutputStream is waiting for ack which 
is ok, but not sure how the DataXCeiver and BlockReceiver should work. It seems 
they're all stuck on socket I/O... Is this a normal? I guess not since the 
usual (~98%) test runs doesn't end up sticking there.... Thanks in advance.

> Intermittent test failure 
> org.apache.hadoop.hdfs.TestCrcCorruption.testCorruptionDuringWrt
> ------------------------------------------------------------------------------------------
>
>                 Key: HDFS-6532
>                 URL: https://issues.apache.org/jira/browse/HDFS-6532
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: datanode, hdfs-client
>    Affects Versions: 2.4.0
>            Reporter: Yongjun Zhang
>            Assignee: Yiqun Lin
>         Attachments: HDFS-6532.001.patch, HDFS-6532.002.patch, 
> PreCommit-HDFS-Build #16770 test - testCorruptionDuringWrt [Jenkins].pdf, 
> TEST-org.apache.hadoop.hdfs.TestCrcCorruption-select_timeout.xml, 
> TEST-org.apache.hadoop.hdfs.TestCrcCorruption.xml, jstack
>
>
> Per https://builds.apache.org/job/Hadoop-Hdfs-trunk/1774/testReport, we had 
> the following failure. Local rerun is successful
> {code}
> Regression
> org.apache.hadoop.hdfs.TestCrcCorruption.testCorruptionDuringWrt
> Failing for the past 1 build (Since Failed#1774 )
> Took 50 sec.
> Error Message
> test timed out after 50000 milliseconds
> Stacktrace
> java.lang.Exception: test timed out after 50000 milliseconds
>       at java.lang.Object.wait(Native Method)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream.waitForAckedSeqno(DFSOutputStream.java:2024)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream.flushInternal(DFSOutputStream.java:2008)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream.close(DFSOutputStream.java:2107)
>       at 
> org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:70)
>       at 
> org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:98)
>       at 
> org.apache.hadoop.hdfs.TestCrcCorruption.testCorruptionDuringWrt(TestCrcCorruption.java:133)
> {code}
> See relevant exceptions in log
> {code}
> 2014-06-14 11:56:15,283 WARN  datanode.DataNode 
> (BlockReceiver.java:verifyChunks(404)) - Checksum error in block 
> BP-1675558312-67.195.138.30-1402746971712:blk_1073741825_1001 from 
> /127.0.0.1:41708
> org.apache.hadoop.fs.ChecksumException: Checksum error: 
> DFSClient_NONMAPREDUCE_-1139495951_8 at 64512 exp: 1379611785 got: -12163112
>       at 
> org.apache.hadoop.util.DataChecksum.verifyChunkedSums(DataChecksum.java:353)
>       at 
> org.apache.hadoop.util.DataChecksum.verifyChunkedSums(DataChecksum.java:284)
>       at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.verifyChunks(BlockReceiver.java:402)
>       at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:537)
>       at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:734)
>       at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:741)
>       at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:124)
>       at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:71)
>       at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:234)
>       at java.lang.Thread.run(Thread.java:662)
> 2014-06-14 11:56:15,285 WARN  datanode.DataNode 
> (BlockReceiver.java:run(1207)) - IOException in BlockReceiver.run(): 
> java.io.IOException: Shutting down writer and responder due to a checksum 
> error in received data. The error response has been sent upstream.
>       at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.sendAckUpstreamUnprotected(BlockReceiver.java:1352)
>       at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.sendAckUpstream(BlockReceiver.java:1278)
>       at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver$PacketResponder.run(BlockReceiver.java:1199)
>       at java.lang.Thread.run(Thread.java:662)
> ...
> {code}



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)

---------------------------------------------------------------------
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