hadoop-hdfs-issues mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From "Hairong Kuang (JIRA)" <j...@apache.org>
Subject [jira] Commented: (HDFS-721) ERROR Block blk_XXX_1030 already exists in state RBW and thus cannot be created
Date Wed, 21 Oct 2009 20:40:59 GMT

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

Hairong Kuang commented on HDFS-721:
------------------------------------

Thanks Stack for the detailed logs. They are very useful for me to investigate the issue.
Was your file's replication factor 3? Were you running your test in a very small cluster?

It seemed to me this was what happened:
1. When the block was created, the initial pipeline has datanodes 139, 140, and 141.
2. 140 failed while the block was being written to.
3. New pipeline was constructed. Only 139 and 141 remained on the pipeline.
4. The block completed. 139 & 141 reported to NN the new replica.
5. NN detetected that the block was under-replicated; therefore scheduled to replicated it
to 140.
6. 140 already has a stale replica of the block; so replication failed.
7. NN scheduled to replicate it to 142. This replication succeeded.

The exception was thrown in step 6. I think this is expected.

> ERROR Block blk_XXX_1030 already exists in state RBW and thus cannot be created
> -------------------------------------------------------------------------------
>
>                 Key: HDFS-721
>                 URL: https://issues.apache.org/jira/browse/HDFS-721
>             Project: Hadoop HDFS
>          Issue Type: Bug
>    Affects Versions: 0.21.0
>         Environment: dfs.support.append=true
> Current branch-0.21 of hdfs, mapreduce, and common. Here is svn info:
> URL: https://svn.apache.org/repos/asf/hadoop/hdfs/branches/branch-0.21
> Repository Root: https://svn.apache.org/repos/asf
> Repository UUID: 13f79535-47bb-0310-9956-ffa450edef68
> Revision: 827883
> Node Kind: directory
> Schedule: normal
> Last Changed Author: szetszwo
> Last Changed Rev: 826906
> Last Changed Date: 2009-10-20 00:16:25 +0000 (Tue, 20 Oct 2009)
>            Reporter: stack
>
> Running some loading tests against hdfs branch-0.21 I got the following:
> {code}
> 2009-10-21 04:57:10,770 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: Receiving
block blk_6345892463926159834_1030 src: /XX.XX.XX.141:53112 dest: /XX.XX.XX.140:51010
> 2009-10-21 04:57:10,771 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: writeBlock
blk_6345892463926159834_1030 received exception org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException:
Block blk_6345892463926159834_1030 already exists in state RBW and thus cannot be created.
> 2009-10-21 04:57:10,771 ERROR org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(XX.XX.XX.140:51010,
storageID=DS-1292310101-XX.XX.XX.140-51010-1256100924816, infoPort=51075, ipcPort=51020):DataXceiver
> org.apache.hadoop.hdfs.server.datanode.ReplicaAlreadyExistsException: Block blk_6345892463926159834_1030
already exists in state RBW and thus cannot be created.
>     at org.apache.hadoop.hdfs.server.datanode.FSDataset.createTemporary(FSDataset.java:1324)
>     at org.apache.hadoop.hdfs.server.datanode.BlockReceiver.<init>(BlockReceiver.java:98)
>     at org.apache.hadoop.hdfs.server.datanode.DataXceiver.opWriteBlock(DataXceiver.java:258)
>     at org.apache.hadoop.hdfs.protocol.DataTransferProtocol$Receiver.opWriteBlock(DataTransferProtocol.java:382)
>     at org.apache.hadoop.hdfs.protocol.DataTransferProtocol$Receiver.processOp(DataTransferProtocol.java:323)
>     at org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:111)
>     at java.lang.Thread.run(Thread.java:619)
> {code}
> On the sender side:
> {code}
> 2009-10-21 04:57:10,740 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(XX.XX.XX.141:51010,
storageID=DS-1870884070-XX.XX.XX.141-51010-1256100925196, infoPort=51075, ipcPort=51020) Starting
thread to transfer block blk_6345892463926159834_1030 to XX.XX.XX.140:51010
> 2009-10-21 04:57:10,770 WARN org.apache.hadoop.hdfs.server.datanode.DataNode: DatanodeRegistration(XX.XX.XX.141:51010,
storageID=DS-1870884070-XX.XX.XX.141-51010-1256100925196, infoPort=51075, ipcPort=51020):Failed
to transfer blk_6345892463926159834_1030 to XX.XX.XX.140:51010 got java.net.SocketException:
Original Exception : java.io.IOException: Connection reset by peer
>     at sun.nio.ch.FileChannelImpl.transferTo0(Native Method)
>     at sun.nio.ch.FileChannelImpl.transferToDirectly(FileChannelImpl.java:415)
>     at sun.nio.ch.FileChannelImpl.transferTo(FileChannelImpl.java:516)
>     at org.apache.hadoop.net.SocketOutputStream.transferToFully(SocketOutputStream.java:199)
>     at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendChunks(BlockSender.java:346)
>     at org.apache.hadoop.hdfs.server.datanode.BlockSender.sendBlock(BlockSender.java:434)
>     at org.apache.hadoop.hdfs.server.datanode.DataNode$DataTransfer.run(DataNode.java:1262)
>     at java.lang.Thread.run(Thread.java:619)
> Caused by: java.io.IOException: Connection reset by peer
>     ... 8 more
> {code}
> The block sequence number, 1030, is one more than that in issue HDFS-720 (same test run
but about 8 seconds between errors.

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


Mime
View raw message