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

Gordon Wang commented on HDFS-6804:
-----------------------------------

After checking the code of Datanode block transferring, I found some race 
condition during transferring the block to the other datanode. And the race 
condition causes the source datanode transfers the wrong checksum of the last 
chunk in replica.

Here is the root cause.
# Datanode DN1 receives transfer block command from NameNode, say, the command 
needs DN1 to transfer block B1 to DataNode DN2.
# DN1 creates a new DataTransfer thread, which is responsible for transferring 
B1 to DN2.
# When DataTransfer thread is created, the replica of B1 is in Finalized state. 
Then, DataTransfer reads replica content and checksum directly from disk, sends 
them to DN2.
# During DataTransfer is sending data to DN2. The block B1 is opened for 
appending. If the last data chunk of B1 is not full, the last checksum will be 
overwritten by the BlockReceiver thread.
# In DataTransfer thread, it records the block length as the length before 
appending. Then, here comes the problem. When DataTransfer thread sends the 
last data chunk to ND2, it reads the checksum of the last chunk from the disk 
and sends the checksum too. But at this time, the last checksum is changed, 
because some more data is appended in the last data chunk.
# When DN2 receives the last data chunk and checksum, it will throw the 
checksum mismatch exception.

The reproduce steps
Prerequisites
# change the code in DataNode.java, sleep a while before sending the block. 
Make these change in DataTransfer.run method.
{code}
        //hack code here
        try {
          LOG.warn("sleep 10 seconds before transfer the block:" + b);
          Thread.sleep(1000L * 10);
        }catch (InterruptedException ie) {
          LOG.error("exception caught.");
        }
        //hack code end

        // send data & checksum
        blockSender.sendBlock(out, unbufOut, null);
{code}

Steps
# Create a HDFS cluster which has 1 NameNode NN and 1 DataNode DN1.
# Create a file F1 whose expected replica factor is 3. Writes some data to the 
file and close it.
# start a new DataNode DN2 to join the cluster.
# grep the log of DN1, when the DataTransfer thread is sleeping, open F1 to 
appends some data, hflush the data to the DN1.

Then, you can find that DN2 throws checksum mismatch exception when receiving 
the last block of file F1.

> race condition between transferring block and appending block causes 
> "Unexpected checksum mismatch exception" 
> --------------------------------------------------------------------------------------------------------------
>
>                 Key: HDFS-6804
>                 URL: https://issues.apache.org/jira/browse/HDFS-6804
>             Project: Hadoop HDFS
>          Issue Type: Bug
>          Components: datanode
>    Affects Versions: 2.2.0
>            Reporter: Gordon Wang
>
> We found some error log in the datanode. like this
> {noformat}
> 2014-07-22 01:49:51,338 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: 
> Ex
> ception for BP-2072804351-192.168.2.104-1406008383435:blk_1073741997_9248
> java.io.IOException: Terminating due to a checksum error.java.io.IOException: 
> Unexpected checksum mismatch while writing 
> BP-2072804351-192.168.2.104-1406008383435:blk_1073741997_9248 from 
> /192.168.2.101:39495
>         at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receivePacket(BlockReceiver.java:536)
>         at 
> org.apache.hadoop.hdfs.server.datanode.BlockReceiver.receiveBlock(BlockReceiver.java:703)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.writeBlock(DataXceiver.java:575)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.opWriteBlock(Receiver.java:115)
>         at 
> org.apache.hadoop.hdfs.protocol.datatransfer.Receiver.processOp(Receiver.java:68)
>         at 
> org.apache.hadoop.hdfs.server.datanode.DataXceiver.run(DataXceiver.java:221)
>         at java.lang.Thread.run(Thread.java:744)
> {noformat}
> While on the source datanode, the log says the block is transmitted.
> {noformat}
> 2014-07-22 01:49:50,805 INFO org.apache.hadoop.hdfs.server.datanode.DataNode: 
> Da
> taTransfer: Transmitted 
> BP-2072804351-192.168.2.104-1406008383435:blk_1073741997
> _9248 (numBytes=16188152) to /192.168.2.103:50010
> {noformat}
> When the destination datanode gets the checksum mismatch, it reports bad 
> block to NameNode and NameNode marks the replica on the source datanode as 
> corrupt. But actually, the replica on the source datanode is valid. Because 
> the replica can pass the checksum verification.
> In all, the replica on the source data is wrongly marked as corrupted.



--
This message was sent by Atlassian JIRA
(v6.2#6252)

Reply via email to