Han Xiao created HDFS-4608:
------------------------------

             Summary: A File could not be recovery, when a block is added to it 
in nn, but not created in dn.
                 Key: HDFS-4608
                 URL: https://issues.apache.org/jira/browse/HDFS-4608
             Project: Hadoop HDFS
          Issue Type: Bug
          Components: hdfs-client
            Reporter: Han Xiao


In such steps. A file would not be recoveried forever.
1. create a file
2. write data and flush
3. terminate the client after the NameNode allocate block for it and before 
client write data to DataNode
4. then shutdown the NN
5. start the NN

then, the file could not be recovered since the new-started NN has no DN to 
report the replica of the block. so any open operation to the file would fail.
It is a very serious problem, since that if it happen to HBase's HLog file, its 
region server would not be started, causing that region server would try to 
open the HLog file with no-endding.

the log of recovering in NN is:
2013-03-18 14:14:19,811 INFO  [IPC Server handler 7 on 8020] recoverLease: 
recover lease [Lease.  Holder: DFSClient_NONMAPREDUCE_154241349_1, 
pendingcreates: 1], src=/testVT/1_5 from client 
DFSClient_NONMAPREDUCE_154241349_1 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.recoverLeaseInternal(FSNamesystem.java:1829)
2013-03-18 14:14:19,811 INFO  [IPC Server handler 7 on 8020] Recovering 
lease=[Lease.  Holder: DFSClient_NONMAPREDUCE_154241349_1, pendingcreates: 1], 
src=/testVT/1_5 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.internalReleaseLease(FSNamesystem.java:2826)
2013-03-18 14:14:19,812 WARN  [IPC Server handler 7 on 8020] BLOCK* 
BlockInfoUnderConstruction.initLeaseRecovery: No blocks found, lease removed. 
org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction.initializeBlockRecovery(BlockInfoUnderConstruction.java:236)
2013-03-18 14:14:19,812 WARN  [IPC Server handler 7 on 8020] DIR* 
NameSystem.internalReleaseLease: File /testVT/1_5 has not been closed. Lease 
recovery is in progress. RecoveryId = 3035 for block 
blk_5356563093942391579_3034{blockUCState=UNDER_RECOVERY, primaryNodeIndex=-1, 
replicas=[]} 
org.apache.hadoop.hdfs.server.namenode.FSNamesystem.internalReleaseLease(FSNamesystem.java:2941)

the log of opening of the file is:
java.io.IOException: Could not obtain the last block locations.
        at 
org.apache.hadoop.hdfs.DFSInputStream.openInfo(DFSInputStream.java:138)
        at org.apache.hadoop.hdfs.DFSInputStream.<init>(DFSInputStream.java:112)
        at org.apache.hadoop.hdfs.DFSClient.open(DFSClient.java:989)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:212)
        at 
org.apache.hadoop.hdfs.DistributedFileSystem.open(DistributedFileSystem.java:75)
        at org.apache.hadoop.fs.FileSystem.open(FileSystem.java:666)

--
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