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

dhruba borthakur commented on HDFS-915:
---------------------------------------

when I first wrote some of this code in 0.16 or so, there were subtle issues on 
how the client detected this failure. This was important because the client had 
to correctly detect which datanode in the pipeline was dead. I am unable to 
recollect that scenario as of now.
But the better way to solve this issue (as done in trunk) is for the client to 
send a ping message periodically, it travels all the way to the last datanode 
in the pipeline... the last datanode then sends an ack back which travels all 
the way back the client. This round-trip tests that all DataStreamers and 
ResponseProcessors are alive and kicking. If the ack for the ping message does 
not arrive, then the the client can start recovery?



> 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
>            Assignee: Todd Lipcon
>         Attachments: local-dn.log
>
>
> 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.

Reply via email to