[ https://issues.apache.org/jira/browse/HDFS-8344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14698054#comment-14698054 ]
Ravi Prakash commented on HDFS-8344: ------------------------------------ Thanks for the report Sean! The recovery time can be modified for tests if you set the timeout super low. e.g. in the unit test {{testLeaseRecoveryWithMissingBlocks}} I use these 2 lines {noformat} cluster.setLeasePeriod(LEASE_PERIOD, LEASE_PERIOD); BlockInfoUnderConstruction.setRecoveryTimeout(1); {noformat} Haohui! By recovery, I meant recovery of a replica (not lease recovery). Please let me know if you think this sequence cannot happen: 1. Client writes data 2. Client dies 3. Datanodes A, B and C (on which data) was written die 4. Lease recovery tries to recover data from Datanodes A and B but fail (because datanodes are dead) 5. Cluster is taken down 6. Datanodes A or B are resurrected 7. The cluster is brought back up after {{RECOVERY_TIMEOUT}} 8. {{FSNamesystem.getShouldForciblyCompleteMissingUCBlock}} returns true because RECOVERY_TIMEOUT has expired 9. Block is forcibly marked complete and the file is labeled as having missing blocks IF Datanode A or B were back, the data would be recovered. The only difference would be that the lease would have been forcefully recovered. > NameNode doesn't recover lease for files with missing blocks > ------------------------------------------------------------ > > Key: HDFS-8344 > URL: https://issues.apache.org/jira/browse/HDFS-8344 > Project: Hadoop HDFS > Issue Type: Bug > Components: namenode > Affects Versions: 2.7.0 > Reporter: Ravi Prakash > Assignee: Ravi Prakash > Fix For: 2.8.0 > > Attachments: HDFS-8344.01.patch, HDFS-8344.02.patch, > HDFS-8344.03.patch, HDFS-8344.04.patch, HDFS-8344.05.patch, > HDFS-8344.06.patch, HDFS-8344.07.patch, HDFS-8344.08.patch, HDFS-8344.09.patch > > > I found another\(?) instance in which the lease is not recovered. This is > reproducible easily on a pseudo-distributed single node cluster > # Before you start it helps if you set. This is not necessary, but simply > reduces how long you have to wait > {code} > public static final long LEASE_SOFTLIMIT_PERIOD = 30 * 1000; > public static final long LEASE_HARDLIMIT_PERIOD = 2 * > LEASE_SOFTLIMIT_PERIOD; > {code} > # Client starts to write a file. (could be less than 1 block, but it hflushed > so some of the data has landed on the datanodes) (I'm copying the client code > I am using. I generate a jar and run it using $ hadoop jar TestHadoop.jar) > # Client crashes. (I simulate this by kill -9 the $(hadoop jar > TestHadoop.jar) process after it has printed "Wrote to the bufferedWriter" > # Shoot the datanode. (Since I ran on a pseudo-distributed cluster, there was > only 1) > I believe the lease should be recovered and the block should be marked > missing. However this is not happening. The lease is never recovered. > The effect of this bug for us was that nodes could not be decommissioned > cleanly. Although we knew that the client had crashed, the Namenode never > released the leases (even after restarting the Namenode) (even months > afterwards). There are actually several other cases too where we don't > consider what happens if ALL the datanodes die while the file is being > written, but I am going to punt on that for another time. -- This message was sent by Atlassian JIRA (v6.3.4#6332)