[ https://issues.apache.org/jira/browse/HDFS-8344?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14697728#comment-14697728 ]
Haohui Mai commented on HDFS-8344: ---------------------------------- If you take down the cluster and bring it back up. All writing pipeline will fail and should fail. bq. I can add one more configuration for the timeout (in addition to the number of retries) This is the exact reason where my previous -1 comes from. This is internal implementation details and I'm very reluctant to make it configurable because (1) it's difficult to determine the right value, and (2) users can easily shoot their foots and cause data loss when these numbers are misconfigured. bq. It feels like we are over-designing now. I disagree. Having only one concept for detecting failures (i.e., time out) is simpler than two (i.e., time out and number of retries). > 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 > > > 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)