You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by jh...@apache.org on 2017/09/28 02:55:01 UTC

[10/50] [abbrv] hadoop git commit: HDFS-12516. Suppress the fsnamesystem lock warning on nn startup. Contributed by Ajay Kumar.

HDFS-12516. Suppress the fsnamesystem lock warning on nn startup. Contributed by Ajay Kumar.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d0b2c585
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d0b2c585
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d0b2c585

Branch: refs/heads/YARN-5734
Commit: d0b2c5850b523a3888b2fadcfcdf6edbed33f221
Parents: 415e5a1
Author: Anu Engineer <ae...@apache.org>
Authored: Sun Sep 24 21:03:06 2017 -0700
Committer: Anu Engineer <ae...@apache.org>
Committed: Sun Sep 24 21:03:06 2017 -0700

----------------------------------------------------------------------
 .../hdfs/server/namenode/FSNamesystem.java      |  7 ++-
 .../hdfs/server/namenode/FSNamesystemLock.java  | 27 +++++++++--
 .../server/namenode/TestFSNamesystemLock.java   | 47 +++++++++++++++++++-
 3 files changed, 76 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b2c585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 967f26b..217df94 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -1095,7 +1095,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (!success) {
         fsImage.close();
       }
-      writeUnlock("loadFSImage");
+      writeUnlock("loadFSImage", true);
     }
     imageLoadComplete();
   }
@@ -1586,6 +1586,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   public void writeUnlock(String opName) {
     this.fsLock.writeUnlock(opName);
   }
+
+  public void writeUnlock(String opName, boolean suppressWriteLockReport) {
+    this.fsLock.writeUnlock(opName, suppressWriteLockReport);
+  }
+
   @Override
   public boolean hasWriteLock() {
     return this.fsLock.isWriteLockedByCurrentThread();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b2c585/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
index 8c60faa..32c7efa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystemLock.java
@@ -207,13 +207,34 @@ class FSNamesystemLock {
     }
   }
 
+  /**
+   * Unlocks FSNameSystem write lock. This internally calls {@link
+   * FSNamesystemLock#writeUnlock(String, boolean)}
+   */
   public void writeUnlock() {
-    writeUnlock(OP_NAME_OTHER);
+    writeUnlock(OP_NAME_OTHER, false);
   }
 
+  /**
+   * Unlocks FSNameSystem write lock. This internally calls {@link
+   * FSNamesystemLock#writeUnlock(String, boolean)}
+   *
+   * @param opName Operation name.
+   */
   public void writeUnlock(String opName) {
-    final boolean needReport = coarseLock.getWriteHoldCount() == 1 &&
-        coarseLock.isWriteLockedByCurrentThread();
+    writeUnlock(opName, false);
+  }
+
+  /**
+   * Unlocks FSNameSystem write lock.
+   *
+   * @param opName Operation name
+   * @param suppressWriteLockReport When false, event of write lock being held
+   * for long time will be logged in logs and metrics.
+   */
+  public void writeUnlock(String opName, boolean suppressWriteLockReport) {
+    final boolean needReport = !suppressWriteLockReport && coarseLock
+        .getWriteHoldCount() == 1 && coarseLock.isWriteLockedByCurrentThread();
     final long currentTimeNanos = timer.monotonicNowNanos();
     final long writeLockIntervalNanos =
         currentTimeNanos - writeLockHeldTimeStampNanos;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d0b2c585/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
index 7f9746d..94e0b33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSNamesystemLock.java
@@ -38,6 +38,7 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.TimeoutException;
 import java.util.regex.Pattern;
+import org.slf4j.LoggerFactory;
 
 import static org.junit.Assert.*;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_FSLOCK_FAIR_KEY;
@@ -347,7 +348,7 @@ public class TestFSNamesystemLock {
 
     fsLock.writeLock();
     timer.advance(1);
-    fsLock.writeUnlock("baz");
+    fsLock.writeUnlock("baz", false);
 
     MetricsRecordBuilder rb = MetricsAsserts.mockMetricsRecordBuilder();
     rates.snapshot(rb, true);
@@ -360,4 +361,48 @@ public class TestFSNamesystemLock {
     assertCounter("FSNWriteLockBazNanosNumOps", 1L, rb);
   }
 
+  /**
+   * Test to suppress FSNameSystem write lock report when it is held for long
+   * time.
+   */
+  @Test(timeout = 45000)
+  public void testFSWriteLockReportSuppressed() throws Exception {
+    final long writeLockReportingThreshold = 1L;
+    final long writeLockSuppressWarningInterval = 10L;
+    Configuration conf = new Configuration();
+    conf.setLong(
+        DFSConfigKeys.DFS_NAMENODE_WRITE_LOCK_REPORTING_THRESHOLD_MS_KEY,
+        writeLockReportingThreshold);
+    conf.setTimeDuration(DFSConfigKeys.DFS_LOCK_SUPPRESS_WARNING_INTERVAL_KEY,
+        writeLockSuppressWarningInterval, TimeUnit.MILLISECONDS);
+
+    final FakeTimer timer = new FakeTimer();
+    final FSNamesystemLock fsnLock = new FSNamesystemLock(conf, null, timer);
+    timer.advance(writeLockSuppressWarningInterval);
+
+    LogCapturer logs = LogCapturer.captureLogs(FSNamesystem.LOG);
+    GenericTestUtils
+        .setLogLevel(LoggerFactory.getLogger(FSNamesystem.class.getName()),
+            org.slf4j.event.Level.INFO);
+
+    // Should trigger the write lock report
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 100);
+    fsnLock.writeUnlock();
+    assertTrue(logs.getOutput().contains(
+        "FSNamesystem write lock held for"));
+
+    logs.clearOutput();
+
+    // Suppress report if the write lock is held for a long time
+    fsnLock.writeLock();
+    timer.advance(writeLockReportingThreshold + 100);
+    fsnLock.writeUnlock("testFSWriteLockReportSuppressed", true);
+    assertFalse(logs.getOutput().contains(GenericTestUtils.getMethodName()));
+    assertFalse(logs.getOutput().contains(
+        "Number of suppressed write-lock reports:"));
+    assertFalse(logs.getOutput().contains(
+        "FSNamesystem write lock held for"));
+  }
+
 }
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-commits-help@hadoop.apache.org