Browse Source

HDFS-14758. Make lease hard limit configurable and reduce the default.
Contributed by hemanthboyina.

Kihwal Lee 5 years ago
parent
commit
9b8a78d97b

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -577,10 +577,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
       } catch (IOException e) {
       } catch (IOException e) {
         // Abort if the lease has already expired.
         // Abort if the lease has already expired.
         final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
         final long elapsed = Time.monotonicNow() - getLastLeaseRenewal();
-        if (elapsed > HdfsConstants.LEASE_HARDLIMIT_PERIOD) {
+        if (elapsed > dfsClientConf.getleaseHardLimitPeriod()) {
           LOG.warn("Failed to renew lease for " + clientName + " for "
           LOG.warn("Failed to renew lease for " + clientName + " for "
               + (elapsed/1000) + " seconds (>= hard-limit ="
               + (elapsed/1000) + " seconds (>= hard-limit ="
-              + (HdfsConstants.LEASE_HARDLIMIT_PERIOD / 1000) + " seconds.) "
+              + (dfsClientConf.getleaseHardLimitPeriod() / 1000) + " seconds.) "
               + "Closing all files being written ...", e);
               + "Closing all files being written ...", e);
           closeAllFilesBeingWritten(true);
           closeAllFilesBeingWritten(true);
         } else {
         } else {

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java

@@ -242,6 +242,9 @@ public interface HdfsClientConfigKeys {
   String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =
   String DFS_PROVIDED_ALIASMAP_INMEMORY_RPC_ADDRESS =
       "dfs.provided.aliasmap.inmemory.dnrpc-address";
       "dfs.provided.aliasmap.inmemory.dnrpc-address";
 
 
+  String DFS_LEASE_HARDLIMIT_KEY = "dfs.namenode.lease-hard-limit-sec";
+  long DFS_LEASE_HARDLIMIT_DEFAULT = 20 * 60;
+
   /**
   /**
    * These are deprecated config keys to client code.
    * These are deprecated config keys to client code.
    */
    */

+ 12 - 0
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java

@@ -151,6 +151,7 @@ public class DfsClientConf {
   private final boolean dataTransferTcpNoDelay;
   private final boolean dataTransferTcpNoDelay;
 
 
   private final boolean deadNodeDetectionEnabled;
   private final boolean deadNodeDetectionEnabled;
+  private final long leaseHardLimitPeriod;
 
 
   public DfsClientConf(Configuration conf) {
   public DfsClientConf(Configuration conf) {
     // The hdfsTimeout is currently the same as the ipc timeout
     // The hdfsTimeout is currently the same as the ipc timeout
@@ -285,6 +286,10 @@ public class DfsClientConf {
         HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY +
         HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY +
         " must be greater than 0.");
         " must be greater than 0.");
     replicaAccessorBuilderClasses = loadReplicaAccessorBuilderClasses(conf);
     replicaAccessorBuilderClasses = loadReplicaAccessorBuilderClasses(conf);
+
+    leaseHardLimitPeriod =
+        conf.getLong(HdfsClientConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
+            HdfsClientConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000;
   }
   }
 
 
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
@@ -618,6 +623,13 @@ public class DfsClientConf {
     return deadNodeDetectionEnabled;
     return deadNodeDetectionEnabled;
   }
   }
 
 
+  /**
+   * @return the leaseHardLimitPeriod
+   */
+  public long getleaseHardLimitPeriod() {
+    return leaseHardLimitPeriod;
+  }
+
   /**
   /**
    * @return the replicaAccessorBuilderClasses
    * @return the replicaAccessorBuilderClasses
    */
    */

+ 0 - 11
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java

@@ -116,17 +116,6 @@ public final class HdfsConstants {
    * lease, another client can preempt the lease.
    * lease, another client can preempt the lease.
    */
    */
   public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
   public static final long LEASE_SOFTLIMIT_PERIOD = 60 * 1000;
-  /**
-   * For a HDFS client to write to a file, a lease is granted; During the lease
-   * period, no other client can write to the file. The writing client can
-   * periodically renew the lease. When the file is closed, the lease is
-   * revoked. The lease duration is bound by a
-   * {@link HdfsConstants#LEASE_SOFTLIMIT_PERIOD soft limit} and this hard
-   * limit. If after the hard limit expires and the client has failed to renew
-   * the lease, HDFS assumes that the client has quit and will automatically
-   * close the file on behalf of the writer, and recover the lease.
-   */
-  public static final long LEASE_HARDLIMIT_PERIOD = 60 * LEASE_SOFTLIMIT_PERIOD;
 
 
   // SafeMode actions
   // SafeMode actions
   public enum SafeModeAction {
   public enum SafeModeAction {

+ 6 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -1741,4 +1741,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   @Deprecated
   @Deprecated
   public static final long    DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
   public static final long    DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT =
       HdfsClientConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT;
       HdfsClientConfigKeys.DFS_CLIENT_KEY_PROVIDER_CACHE_EXPIRY_DEFAULT;
+
+  public static final String DFS_LEASE_HARDLIMIT_KEY =
+      HdfsClientConfigKeys.DFS_LEASE_HARDLIMIT_KEY;
+  public static final long DFS_LEASE_HARDLIMIT_DEFAULT =
+      HdfsClientConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT;
+
 }
 }

+ 6 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java

@@ -41,7 +41,9 @@ import com.google.common.collect.Lists;
 
 
 import org.apache.commons.lang3.StringUtils;
 import org.apache.commons.lang3.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
 import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
 import org.apache.hadoop.hdfs.protocol.OpenFileEntry;
@@ -84,7 +86,7 @@ public class LeaseManager {
       .getName());
       .getName());
   private final FSNamesystem fsnamesystem;
   private final FSNamesystem fsnamesystem;
   private long softLimit = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
   private long softLimit = HdfsConstants.LEASE_SOFTLIMIT_PERIOD;
-  private long hardLimit = HdfsConstants.LEASE_HARDLIMIT_PERIOD;
+  private long hardLimit;
   static final int INODE_FILTER_WORKER_COUNT_MAX = 4;
   static final int INODE_FILTER_WORKER_COUNT_MAX = 4;
   static final int INODE_FILTER_WORKER_TASK_MIN = 512;
   static final int INODE_FILTER_WORKER_TASK_MIN = 512;
   private long lastHolderUpdateTime;
   private long lastHolderUpdateTime;
@@ -112,7 +114,10 @@ public class LeaseManager {
   private volatile boolean shouldRunMonitor;
   private volatile boolean shouldRunMonitor;
 
 
   LeaseManager(FSNamesystem fsnamesystem) {
   LeaseManager(FSNamesystem fsnamesystem) {
+    Configuration conf = new Configuration();
     this.fsnamesystem = fsnamesystem;
     this.fsnamesystem = fsnamesystem;
+    this.hardLimit = conf.getLong(DFSConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
+        DFSConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000;
     updateInternalLeaseHolder();
     updateInternalLeaseHolder();
   }
   }
 
 

+ 8 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml

@@ -5745,4 +5745,12 @@
       octal or symbolic.
       octal or symbolic.
     </description>
     </description>
   </property>
   </property>
+
+  <property>
+    <name>dfs.namenode.lease-hard-limit-sec</name>
+    <value>1200</value>
+    <description>
+      Determines the namenode automatic lease recovery interval in seconds.
+    </description>
+  </property>
 </configuration>
 </configuration>

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileAppend4.java

@@ -40,7 +40,6 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
@@ -107,7 +106,9 @@ public class TestFileAppend4 {
 
 
     // set the soft limit to be 1 second so that the
     // set the soft limit to be 1 second so that the
     // namenode triggers lease recovery upon append request
     // namenode triggers lease recovery upon append request
-    cluster.setLeasePeriod(1000, HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+    cluster.setLeasePeriod(1,
+        conf.getLong(DFSConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
+            DFSConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT));
 
 
     // Trying recovery
     // Trying recovery
     int tries = 60;
     int tries = 60;

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java

@@ -111,9 +111,10 @@ public class TestLease {
         Assert.fail("Write failed.");
         Assert.fail("Write failed.");
       }
       }
 
 
+      long hardlimit = conf.getLong(DFSConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
+          DFSConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000;
       // make it look like the hard limit has been exceeded.
       // make it look like the hard limit has been exceeded.
-      dfs.lastLeaseRenewal = Time.monotonicNow()
-      - HdfsConstants.LEASE_HARDLIMIT_PERIOD - 1000;
+      dfs.lastLeaseRenewal = Time.monotonicNow() - hardlimit - 1000;
       dfs.renewLease();
       dfs.renewLease();
 
 
       // this should not work.
       // this should not work.

+ 3 - 3
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLeaseRecovery2.java

@@ -407,10 +407,10 @@ public class TestLeaseRecovery2 {
     Map<String, String []> u2g_map = new HashMap<String, String []>(1);
     Map<String, String []> u2g_map = new HashMap<String, String []>(1);
     u2g_map.put(fakeUsername, new String[] {fakeGroup});
     u2g_map.put(fakeUsername, new String[] {fakeGroup});
     DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
     DFSTestUtil.updateConfWithFakeGroupMapping(conf, u2g_map);
-
+    long hardlimit = conf.getLong(DFSConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
+        DFSConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000;
     // Reset default lease periods
     // Reset default lease periods
-    cluster.setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
-                           HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+    cluster.setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD, hardlimit);
     //create a file
     //create a file
     // create a random file name
     // create a random file name
     String filestr = "/foo" + AppendTestUtil.nextInt();
     String filestr = "/foo" + AppendTestUtil.nextInt();

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java

@@ -659,7 +659,8 @@ public class TestFileTruncate {
 
 
     NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
     NameNodeAdapter.getLeaseManager(cluster.getNamesystem())
         .setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
         .setLeasePeriod(HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
-            HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+            conf.getLong(DFSConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
+                DFSConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000);
 
 
     checkFullFile(p, newLength, contents);
     checkFullFile(p, newLength, contents);
     fs.delete(p, false);
     fs.delete(p, false);

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestINodeFileUnderConstructionWithSnapshot.java

@@ -304,7 +304,8 @@ public class TestINodeFileUnderConstructionWithSnapshot {
       }
       }
     } finally {
     } finally {
       NameNodeAdapter.setLeasePeriod(fsn, HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
       NameNodeAdapter.setLeasePeriod(fsn, HdfsConstants.LEASE_SOFTLIMIT_PERIOD,
-          HdfsConstants.LEASE_HARDLIMIT_PERIOD);
+          conf.getLong(DFSConfigKeys.DFS_LEASE_HARDLIMIT_KEY,
+              DFSConfigKeys.DFS_LEASE_HARDLIMIT_DEFAULT) * 1000);
     }
     }
   }
   }
 }
 }