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

Tsz Wo (Nicholas), SZE commented on HDFS-4600:
----------------------------------------------

Hi Roman, append and create are different in the sense that there is existing 
data in before append.  The existing data were already persisted in HDFS 
earlier.  The replace-datanode-on-failure feature is to prevent data lost.  For 
append, it makes sure there are enough datanodes to protect the existing data.  
It is very useful for slow appender.  The feature can be disabled by setting 
dfs.client.block.write.replace-datanode-on-failure.enable if it is undesirable.

This JIRA is similar to HDFS-3091.
                
> HDFS file append failing in multinode cluster
> ---------------------------------------------
>
>                 Key: HDFS-4600
>                 URL: https://issues.apache.org/jira/browse/HDFS-4600
>             Project: Hadoop HDFS
>          Issue Type: Bug
>    Affects Versions: 2.0.3-alpha
>            Reporter: Roman Shaposhnik
>            Priority: Blocker
>             Fix For: 2.0.4-alpha
>
>         Attachments: core-site.xml, hdfs-site.xml, X.java
>
>
> NOTE: the following only happens in a fully distributed setup (core-site.xml 
> and hdfs-site.xml are attached)
> Steps to reproduce:
> {noformat}
> $ javac -cp /usr/lib/hadoop/client/\* X.java
> $ echo aaaaa > a.txt
> $ hadoop fs -ls /tmp/a.txt
> ls: `/tmp/a.txt': No such file or directory
> $ HADOOP_CLASSPATH=`pwd` hadoop X /tmp/a.txt
> 13/03/13 16:05:14 WARN hdfs.DFSClient: DataStreamer Exception
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[10.10.37.16:50010, 10.80.134.126:50010], 
> original=[10.10.37.16:50010, 10.80.134.126:50010]). The current failed 
> datanode replacement policy is DEFAULT, and a client may configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.findNewDatanode(DFSOutputStream.java:793)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:858)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:964)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:470)
> Exception in thread "main" java.io.IOException: Failed to replace a bad 
> datanode on the existing pipeline due to no more good datanodes being 
> available to try. (Nodes: current=[10.10.37.16:50010, 10.80.134.126:50010], 
> original=[10.10.37.16:50010, 10.80.134.126:50010]). The current failed 
> datanode replacement policy is DEFAULT, and a client may configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.findNewDatanode(DFSOutputStream.java:793)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:858)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:964)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:470)
> 13/03/13 16:05:14 ERROR hdfs.DFSClient: Failed to close file /tmp/a.txt
> java.io.IOException: Failed to replace a bad datanode on the existing 
> pipeline due to no more good datanodes being available to try. (Nodes: 
> current=[10.10.37.16:50010, 10.80.134.126:50010], 
> original=[10.10.37.16:50010, 10.80.134.126:50010]). The current failed 
> datanode replacement policy is DEFAULT, and a client may configure this via 
> 'dfs.client.block.write.replace-datanode-on-failure.policy' in its 
> configuration.
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.findNewDatanode(DFSOutputStream.java:793)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:858)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:964)
>       at 
> org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:470)
> {noformat}
> Given that the file actually does get created:
> {noformat}
> $ hadoop fs -ls /tmp/a.txt
> Found 1 items
> -rw-r--r--   3 root hadoop          6 2013-03-13 16:05 /tmp/a.txt
> {noformat}
> this feels like a regression in APPEND's functionality.

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators
For more information on JIRA, see: http://www.atlassian.com/software/jira

Reply via email to