Uploaded image for project: 'Hadoop HDFS'
  1. Hadoop HDFS
  2. HDFS-4608

A File could not be recovery, when a block is added to it in nn, but not created in dn.

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Duplicate
    • None
    • None
    • hdfs-client
    • None

    Description

      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 writes 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(not write data to any DNs) 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:
      2013-03-18 15:36:02,814 DEBUG ipc.ProtobufRpcEngine (ProtobufRpcEngine.java:invoke(192)) - Call: getBlockLocations 12
      2013-03-18 15:36:02,814 DEBUG hdfs.DFSClient (DFSInputStream.java:fetchLocatedBlocksAndGetLastBlockLength(154)) - newInfo = LocatedBlocks{
      fileLength=0
      underConstruction=true
      blocks=[LocatedBlock{BP-myhacluster-25656:blk_5356563093942391579_3034; getBlockSize()=0; corrupt=false; offset=0; locs=[]}]
      lastLocatedBlock=LocatedBlock{BP-myhacluster-25656:blk_5356563093942391579_3034; getBlockSize()=0; corrupt=false; offset=0; locs=[]}
      isLastBlockComplete=false}
      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)

      Attachments

        Activity

          People

            Unassigned Unassigned
            yians Han Xiao
            Votes:
            0 Vote for this issue
            Watchers:
            4 Start watching this issue

            Dates

              Created:
              Updated:
              Resolved: