Brahma Reddy Battula created HDFS-3436:
------------------------------------------

             Summary: Append to file is failing when one of the datanode where 
the block present is down.
                 Key: HDFS-3436
                 URL: https://issues.apache.org/jira/browse/HDFS-3436
             Project: Hadoop HDFS
          Issue Type: Bug
          Components: data-node
    Affects Versions: 2.0.0
            Reporter: Brahma Reddy Battula


Scenario:
=========

1. Cluster with 4 DataNodes.
2. Written file to 3 DNs, DN1->DN2->DN3
3. Stopped DN3,
Now Append to file is failing due to addDatanode2ExistingPipeline is failed.

 *CLinet Trace* 
{noformat}
2012-04-24 22:06:09,947 INFO  hdfs.DFSClient 
(DFSOutputStream.java:createBlockOutputStream(1063)) - Exception in 
createBlockOutputStream
java.io.IOException: Bad connect ack with firstBadLink as *******:50010
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.createBlockOutputStream(DFSOutputStream.java:1053)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:943)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:461)
2012-04-24 22:06:09,947 WARN  hdfs.DFSClient 
(DFSOutputStream.java:setupPipelineForAppendOrRecovery(916)) - Error Recovery 
for block BP-1023239-10.18.40.233-1335275282109:blk_296651611851855249_1253 in 
pipeline *****:50010, ******:50010, *****:50010: bad datanode ******:50010
2012-04-24 22:06:10,072 WARN  hdfs.DFSClient (DFSOutputStream.java:run(549)) - 
DataStreamer Exception
java.io.EOFException: Premature EOF: no length prefix available
        at 
org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed(HdfsProtoUtil.java:162)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.transfer(DFSOutputStream.java:866)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:843)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:934)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:461)
2012-04-24 22:06:10,072 WARN  hdfs.DFSClient 
(DFSOutputStream.java:hflush(1515)) - Error while syncing
java.io.EOFException: Premature EOF: no length prefix available
        at 
org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed(HdfsProtoUtil.java:162)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.transfer(DFSOutputStream.java:866)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:843)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:934)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:461)
java.io.EOFException: Premature EOF: no length prefix available
        at 
org.apache.hadoop.hdfs.protocol.HdfsProtoUtil.vintPrefixed(HdfsProtoUtil.java:162)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.transfer(DFSOutputStream.java:866)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.addDatanode2ExistingPipeline(DFSOutputStream.java:843)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.setupPipelineForAppendOrRecovery(DFSOutputStream.java:934)
        at 
org.apache.hadoop.hdfs.DFSOutputStream$DataStreamer.run(DFSOutputStream.java:461)
{noformat}

 *DataNode Trace*  

{noformat}

2012-05-17 15:39:12,261 ERROR datanode.DataNode (DataXceiver.java:run(193)) - 
host0.foo.com:49744:DataXceiver error processing TRANSFER_BLOCK operation  src: 
/127.0.0.1:49811 dest: /127.0.0.1:49744
java.io.IOException: 
BP-2001850558-10.18.47.190-1337249347060:blk_-8165642083860293107_1002 is 
neither a RBW nor a Finalized, r=ReplicaBeingWritten, 
blk_-8165642083860293107_1003, RBW
  getNumBytes()     = 1024
  getBytesOnDisk()  = 1024
  getVisibleLength()= 1024
  getVolume()       = 
E:\MyWorkSpace\branch-2\Test\build\test\data\dfs\data\data1\current
  getBlockFile()    = 
E:\MyWorkSpace\branch-2\Test\build\test\data\dfs\data\data1\current\BP-2001850558-10.18.47.190-1337249347060\current\rbw\blk_-8165642083860293107
  bytesAcked=1024
  bytesOnDisk=102
at 
org.apache.hadoop.hdfs.server.datanode.DataNode.transferReplicaForPipelineRecovery(DataNode.java:2038)
        at 
org.apache.hadoop.hdfs.server.datanode.DataXceiver.transferBlock(DataXceiver.java:525)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opTransferBlock(Receiver.java:114)
        at 
org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:78)
        at 
org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:189)
        at java.lang.Thread.run(Unknown Source)
{noformat}

--
This message is automatically generated by JIRA.
If you think it was sent incorrectly, please contact your JIRA administrators: 
https://issues.apache.org/jira/secure/ContactAdministrators!default.jspa
For more information on JIRA, see: http://www.atlassian.com/software/jira

        

Reply via email to