فهرست منبع

HDFS-10798. Make the threshold of reporting FSNamesystem lock contention configurable. Contributed by Erik Krogen.

Zhe Zhang 8 سال پیش
والد
کامیت
407b519fb1

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

@@ -407,6 +407,11 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
   public static final long
       DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT = 25;
 
+  // Threshold for how long a write lock must be held for the event to be logged
+  public static final String  DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY =
+      "dfs.namenode.write-lock-reporting-threshold-ms";
+  public static final long    DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT = 1000L;
+
   public static final String  DFS_UPGRADE_DOMAIN_FACTOR = "dfs.namenode.upgrade.domain.factor";
   public static final int DFS_UPGRADE_DOMAIN_FACTOR_DEFAULT = DFS_REPLICATION_DEFAULT;
 

+ 8 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -71,6 +71,8 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RESOURCE_CHECK_INTERVAL_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY;
+import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_EXPIRYTIME_MILLIS_KEY;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_RETRY_CACHE_HEAP_PERCENT_DEFAULT;
@@ -819,6 +821,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_KEY,
           DFS_NAMENODE_MAX_LOCK_HOLD_TO_RELEASE_LEASE_MS_DEFAULT);
 
+      this.writeLockReportingThreshold = conf.getLong(
+          DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+          DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_DEFAULT);
+
       // For testing purposes, allow the DT secret manager to be started regardless
       // of whether security is enabled.
       alwaysUseDelegationTokensForTests = conf.getBoolean(
@@ -1498,7 +1504,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   /** Threshold (ms) for long holding write lock report. */
-  static final short WRITELOCK_REPORTING_THRESHOLD = 1000;
+  private long writeLockReportingThreshold;
   /** Last time stamp for write lock. Keep the longest one for multi-entrance.*/
   private long writeLockHeldTimeStamp;
 
@@ -1532,7 +1538,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     this.fsLock.writeLock().unlock();
 
-    if (needReport && writeLockInterval >= WRITELOCK_REPORTING_THRESHOLD) {
+    if (needReport && writeLockInterval >= this.writeLockReportingThreshold) {
       LOG.info("FSNamesystem write lock held for " + writeLockInterval +
           " ms via\n" + StringUtils.getStackTrace(Thread.currentThread()));
     }

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

@@ -2614,6 +2614,15 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.write-lock-reporting-threshold-ms</name>
+  <value>1000</value>
+  <description>When a write lock is held on the namenode for a long time,
+    this will be logged as the lock is released. This sets how long the
+    lock must be held for logging to occur.
+  </description>
+</property>
+
 <property>
   <name>dfs.namenode.startup.delay.block.deletion.sec</name>
   <value>0</value>

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystem.java

@@ -290,7 +290,10 @@ public class TestFSNamesystem {
    */
   @Test(timeout=45000)
   public void testFSLockLongHoldingReport() throws Exception {
+    final long writeLockReportingThreshold = 100L;
     Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        writeLockReportingThreshold);
     FSImage fsImage = Mockito.mock(FSImage.class);
     FSEditLog fsEditLog = Mockito.mock(FSEditLog.class);
     Mockito.when(fsImage.getEditLog()).thenReturn(fsEditLog);
@@ -301,32 +304,32 @@ public class TestFSNamesystem {
 
     // Don't report if the write lock is held for a short time
     fsn.writeLock();
-    Thread.sleep(FSNamesystem.WRITELOCK_REPORTING_THRESHOLD / 2);
+    Thread.sleep(writeLockReportingThreshold / 2);
     fsn.writeUnlock();
     assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
 
 
     // Report if the write lock is held for a long time
     fsn.writeLock();
-    Thread.sleep(FSNamesystem.WRITELOCK_REPORTING_THRESHOLD + 100);
+    Thread.sleep(writeLockReportingThreshold + 10);
     logs.clearOutput();
     fsn.writeUnlock();
     assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName()));
 
     // Report if the write lock is held (interruptibly) for a long time
     fsn.writeLockInterruptibly();
-    Thread.sleep(FSNamesystem.WRITELOCK_REPORTING_THRESHOLD + 100);
+    Thread.sleep(writeLockReportingThreshold + 10);
     logs.clearOutput();
     fsn.writeUnlock();
     assertTrue(logs.getOutput().contains(GenericTestUtils.getMethodName()));
 
     // Report if it's held for a long time when re-entering write lock
     fsn.writeLock();
-    Thread.sleep(FSNamesystem.WRITELOCK_REPORTING_THRESHOLD / 2 + 1);
+    Thread.sleep(writeLockReportingThreshold/ 2 + 1);
     fsn.writeLockInterruptibly();
-    Thread.sleep(FSNamesystem.WRITELOCK_REPORTING_THRESHOLD / 2 + 1);
+    Thread.sleep(writeLockReportingThreshold / 2 + 1);
     fsn.writeLock();
-    Thread.sleep(FSNamesystem.WRITELOCK_REPORTING_THRESHOLD / 2);
+    Thread.sleep(writeLockReportingThreshold / 2);
     logs.clearOutput();
     fsn.writeUnlock();
     assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));