You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-dev@hadoop.apache.org by "Raghu Angadi (JIRA)" <ji...@apache.org> on 2008/04/11 03:00:06 UTC

[jira] Commented: (HADOOP-3132) DFS writes stuck occationally

    [ https://issues.apache.org/jira/browse/HADOOP-3132?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=12587824#action_12587824 ] 

Raghu Angadi commented on HADOOP-3132:
--------------------------------------

Runping, Dhruba and I have been looking at this. findings till now are strange. It is strange enough to be a kernel bug. Here a client is writing a block to three datanodes : DN-1, DN-2, and DN-3 in this order.

In brief, what happens is that DN-3 gets stuck while reading data from DN-2. But netstat on DN-3 shows that recv buffer for socket from DN-2 is full. The stacktrace of the stuck thread on DN-3 is shown below (*). So we suspected may be there is a bug in JRE with non-blocking sockets or kernel. 

Even though these sockets are running with large timeout values (not the default 10min), at around 16min after the pipeline is stuck like this,   DN-2 gets "Connection timeout out" error while it is reading from the socket for ack for the last packet from DN-3. DN-3 keeps sending the 'keep-alive" every 30 seconds and DN-2 receives them every 30 seconds till then. If there is no problem at the kernel or jre, there is no reason for this read to get such an error (unless there is some TCP timeout we don't know), note that the block receive thread at DN-2 w. At this point write pipeline gets broken. This pipeline should have stayed stuck forever ("keep-alive" messages keep coming).

Another option tied is to use regular sockets on DataNodes using patch from HADOOP-3124. The same thing happened again. (This time the pipeline did not recover properly and the task failed.. I will file another jira for it).

Next thing to try is to select() on the write socket with 10 second timeout at a time (user specified timeout will still be obeyed.. but the thread wakes up every 10 seconds) and hope for better.

(*) : Stacktrace on DN-3 : {noformat}
"org.apache.hadoop.dfs.DataNode$DataXceiver@16fcc4" daemon prio=10 tid=0x08567400 nid=0x7d7 runnable [0xb1072000..0xb10730a0]
   java.lang.Thread.State: RUNNABLE
        at sun.nio.ch.EPollArrayWrapper.epollWait(Native Method)
        at sun.nio.ch.EPollArrayWrapper.poll(EPollArrayWrapper.java:184)
        at sun.nio.ch.EPollSelectorImpl.doSelect(EPollSelectorImpl.java:65)
        at sun.nio.ch.SelectorImpl.lockAndDoSelect(SelectorImpl.java:69)
        - locked <0xb64d54d8> (a sun.nio.ch.Util$1)
        - locked <0xb64d54c8> (a java.util.Collections$UnmodifiableSet)
        - locked <0xb64d52c0> (a sun.nio.ch.EPollSelectorImpl)
        at sun.nio.ch.SelectorImpl.select(SelectorImpl.java:80)
        at org.apache.hadoop.net.SocketIOWithTimeout$SelectorPool.select(SocketIOWithTimeout.java:237)
        at org.apache.hadoop.net.SocketIOWithTimeout.doIO(SocketIOWithTimeout.java:155)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:149)
        at org.apache.hadoop.net.SocketInputStream.read(SocketInputStream.java:122)
        at java.io.BufferedInputStream.fill(BufferedInputStream.java:218)
        at java.io.BufferedInputStream.read1(BufferedInputStream.java:258)
        at java.io.BufferedInputStream.read(BufferedInputStream.java:317)
        - locked <0xb66b3858> (a java.io.BufferedInputStream)
        at java.io.DataInputStream.readFully(DataInputStream.java:178)
        at org.apache.hadoop.dfs.DataNode$BlockReceiver.receiveChunk(DataNode.java:2257)
        at org.apache.hadoop.dfs.DataNode$BlockReceiver.receivePacket(DataNode.java:2398)
        at org.apache.hadoop.dfs.DataNode$BlockReceiver.receiveBlock(DataNode.java:2463)
        at org.apache.hadoop.dfs.DataNode$DataXceiver.writeBlock(DataNode.java:1192)
        at org.apache.hadoop.dfs.DataNode$DataXceiver.run(DataNode.java:964)
        at java.lang.Thread.run(Thread.java:619)
{noformat}




 




> DFS writes stuck occationally
> -----------------------------
>
>                 Key: HADOOP-3132
>                 URL: https://issues.apache.org/jira/browse/HADOOP-3132
>             Project: Hadoop Core
>          Issue Type: Bug
>          Components: dfs
>            Reporter: Runping Qi
>            Assignee: Raghu Angadi
>            Priority: Blocker
>             Fix For: 0.17.0
>
>
> This problem happens in 0.17 trunk
> As reported in hadoop-3124,
> I saw reducers waited 10 minutes for writing data to dfs and got timeout.
> The client retries again and timeouted after another 19 minutes.
> During the period of write stuck, all the nodes in the data node pipeline were functioning fine.
> The system load was normal.
> I don't believe this was due to slow network cards/disk drives or overloaded machines.
> I believe this and hadoop-3033 are related somehow.

-- 
This message is automatically generated by JIRA.
-
You can reply to this email to add a comment to the issue online.