This is an automated email from the ASF dual-hosted git repository.

zanderxu pushed a commit to branch HDFS-17384
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit ca902d1212e77d3855b696124b9ca648b730f3d0
Author: ZanderXu <zande...@apache.org>
AuthorDate: Mon Mar 4 18:00:18 2024 +0800

    HDFS-17387. [FGL] Abstract the configuration locking mode (#6572)
    
    * HDFS-17387. [FGL] Abstract the configurable locking mode
---
 .../java/org/apache/hadoop/hdfs/DFSConfigKeys.java |   6 +
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  |  96 +++++++----
 .../hdfs/server/namenode/FSNamesystemLock.java     |  12 +-
 .../hdfs/server/namenode/fgl/FSNLockManager.java   | 184 +++++++++++++++++++++
 .../server/namenode/fgl/FSNamesystemLockMode.java  |  24 +++
 .../namenode/fgl/GlobalFSNamesystemLock.java       | 149 +++++++++++++++++
 .../hdfs/server/namenode/fgl/package-info.java     |  18 ++
 .../java/org/apache/hadoop/hdfs/util/RwLock.java   |  78 +++++++--
 .../src/main/resources/hdfs-default.xml            |   9 +
 9 files changed, 525 insertions(+), 51 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index f92a2ad56581..337980ac5200 100755
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -29,6 +29,8 @@ import 
org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFau
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import 
org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker;
 import 
org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.ReservedSpaceCalculator;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNLockManager;
+import org.apache.hadoop.hdfs.server.namenode.fgl.GlobalFSNamesystemLock;
 import org.apache.hadoop.hdfs.web.URLConnectionFactory;
 import org.apache.hadoop.http.HttpConfig;
 
@@ -1051,6 +1053,10 @@ public class DFSConfigKeys extends 
CommonConfigurationKeys {
   public static final String  DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY = 
"dfs.namenode.inode.attributes.provider.class";
   public static final String  
DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_KEY = 
"dfs.namenode.inode.attributes.provider.bypass.users";
   public static final String  
DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_BYPASS_USERS_DEFAULT = "";
+  public static final String  DFS_NAMENODE_LOCK_MODEL_PROVIDER_KEY =
+      "dfs.namenode.lock.model.provider.class";
+  public static final Class<? extends FSNLockManager>  
DFS_NAMENODE_LOCK_MODEL_PROVIDER_DEFAULT =
+      GlobalFSNamesystemLock.class;
 
   public static final String  DFS_DATANODE_BP_READY_TIMEOUT_KEY = 
"dfs.datanode.bp-ready.timeout";
   public static final long    DFS_DATANODE_BP_READY_TIMEOUT_DEFAULT = 20;
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 8470eac05f83..e5372bf1981b 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
@@ -34,6 +34,8 @@ import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_BLOCK_SIZE_KEY;
 import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_WITH_REMOTE_PORT_DEFAULT;
 import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_AUDIT_LOG_WITH_REMOTE_PORT_KEY;
+import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_MODEL_PROVIDER_DEFAULT;
+import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LOCK_MODEL_PROVIDER_KEY;
 import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_ENABLED_DEFAULT;
 import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_PERMISSIONS_SUPERUSER_ONLY_DEFAULT;
 import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_STORAGE_POLICY_PERMISSIONS_SUPERUSER_ONLY_KEY;
@@ -96,6 +98,7 @@ import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LI
 import static 
org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_SNAPSHOT_DIFF_LISTING_LIMIT_DEFAULT;
 import static org.apache.hadoop.hdfs.DFSUtil.isParentEntry;
 
+import java.lang.reflect.Constructor;
 import java.nio.charset.StandardCharsets;
 import java.util.concurrent.atomic.AtomicLong;
 
@@ -114,6 +117,8 @@ import static 
org.apache.hadoop.ha.HAServiceProtocol.HAServiceState.OBSERVER;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicyInfo;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNLockManager;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.thirdparty.com.google.common.collect.Maps;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotDeletionGc;
 import org.apache.hadoop.thirdparty.protobuf.ByteString;
@@ -621,7 +626,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   private final int numCommittedAllowed;
 
   /** Lock to protect FSNamesystem. */
-  private final FSNamesystemLock fsLock;
+  private final FSNLockManager fsLock;
 
   /** 
    * Checkpoint lock to protect FSNamesystem modification on standby NNs.
@@ -871,7 +876,10 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     this.contextFieldSeparator =
         conf.get(HADOOP_CALLER_CONTEXT_SEPARATOR_KEY,
             HADOOP_CALLER_CONTEXT_SEPARATOR_DEFAULT);
-    fsLock = new FSNamesystemLock(conf, detailedLockHoldTimeMetrics);
+    Class<? extends FSNLockManager> lockKlass = conf.getClass(
+        DFS_NAMENODE_LOCK_MODEL_PROVIDER_KEY, 
DFS_NAMENODE_LOCK_MODEL_PROVIDER_DEFAULT,
+        FSNLockManager.class);
+    fsLock = createLock(lockKlass, conf, detailedLockHoldTimeMetrics);
     cpLock = new ReentrantLock();
 
     this.fsImage = fsImage;
@@ -1077,6 +1085,18 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     }
   }
 
+  private <T> T createLock(Class<T> theClass, Configuration conf,
+      MutableRatesWithAggregation mutableRatesMetrics) {
+    try {
+      Constructor<T> meth = theClass.getDeclaredConstructor(
+          Configuration.class, MutableRatesWithAggregation.class);
+      meth.setAccessible(true);
+      return meth.newInstance(conf, mutableRatesMetrics);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+  }
+
   private static void checkForAsyncLogEnabledByOldConfigs(Configuration conf) {
     // dfs.namenode.audit.log.async is no longer in use. Use log4j properties 
instead.
     if (conf.getBoolean("dfs.namenode.audit.log.async", false)) {
@@ -1787,70 +1807,74 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   }
 
   @Override
-  public void readLock() {
-    this.fsLock.readLock();
-  }
-
-  @Override
-  public void readLockInterruptibly() throws InterruptedException {
-    this.fsLock.readLockInterruptibly();
+  public void readLock(FSNamesystemLockMode lockMode) {
+    this.fsLock.readLock(lockMode);
   }
 
   @Override
-  public void readUnlock() {
-    this.fsLock.readUnlock();
+  public void readLockInterruptibly(FSNamesystemLockMode lockMode) throws 
InterruptedException {
+    this.fsLock.readLockInterruptibly(lockMode);
   }
 
   @Override
-  public void readUnlock(String opName) {
-    this.fsLock.readUnlock(opName);
+  public void readUnlock(FSNamesystemLockMode lockMode, String opName) {
+    this.fsLock.readUnlock(lockMode, opName);
   }
 
   public void readUnlock(String opName,
       Supplier<String> lockReportInfoSupplier) {
-    this.fsLock.readUnlock(opName, lockReportInfoSupplier);
+    readUnlock(FSNamesystemLockMode.GLOBAL, opName, lockReportInfoSupplier);
   }
 
-  @Override
-  public void writeLock() {
-    this.fsLock.writeLock();
+  public void readUnlock(FSNamesystemLockMode lockMode, String opName,
+      Supplier<String> lockReportInfoSupplier) {
+    this.fsLock.readUnlock(lockMode, opName, lockReportInfoSupplier);
   }
 
   @Override
-  public void writeLockInterruptibly() throws InterruptedException {
-    this.fsLock.writeLockInterruptibly();
+  public void writeLock(FSNamesystemLockMode lockMode) {
+    this.fsLock.writeLock(lockMode);
   }
 
   @Override
-  public void writeUnlock() {
-    this.fsLock.writeUnlock();
+  public void writeLockInterruptibly(FSNamesystemLockMode lockMode) throws 
InterruptedException {
+    this.fsLock.writeLockInterruptibly(lockMode);
   }
 
   @Override
-  public void writeUnlock(String opName) {
-    this.fsLock.writeUnlock(opName);
+  public void writeUnlock(FSNamesystemLockMode lockMode, String opName) {
+    this.fsLock.writeUnlock(lockMode, opName);
   }
 
   public void writeUnlock(String opName, boolean suppressWriteLockReport) {
-    this.fsLock.writeUnlock(opName, suppressWriteLockReport);
+    writeUnlock(FSNamesystemLockMode.GLOBAL, opName, suppressWriteLockReport);
+  }
+
+  public void writeUnlock(FSNamesystemLockMode lockMode, String opName,
+      boolean suppressWriteLockReport) {
+    this.fsLock.writeUnlock(lockMode, opName, suppressWriteLockReport);
+  }
+
+  public void writeUnlock(String opName, Supplier<String> 
lockReportInfoSupplier) {
+    writeUnlock(FSNamesystemLockMode.GLOBAL, opName, lockReportInfoSupplier);
   }
 
-  public void writeUnlock(String opName,
+  public void writeUnlock(FSNamesystemLockMode lockMode, String opName,
       Supplier<String> lockReportInfoSupplier) {
-    this.fsLock.writeUnlock(opName, lockReportInfoSupplier);
+    this.fsLock.writeUnlock(lockMode, opName, lockReportInfoSupplier);
   }
 
   @Override
-  public boolean hasWriteLock() {
-    return this.fsLock.isWriteLockedByCurrentThread();
+  public boolean hasWriteLock(FSNamesystemLockMode lockMode) {
+    return this.fsLock.hasWriteLock(lockMode);
   }
   @Override
-  public boolean hasReadLock() {
-    return this.fsLock.getReadHoldCount() > 0 || hasWriteLock();
+  public boolean hasReadLock(FSNamesystemLockMode lockMode) {
+    return this.fsLock.hasReadLock(lockMode);
   }
 
   public int getReadHoldCount() {
-    return this.fsLock.getReadHoldCount();
+    return this.fsLock.getReadHoldCount(FSNamesystemLockMode.GLOBAL);
   }
 
   /** Lock the checkpoint lock */
@@ -4931,21 +4955,21 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   @Metric({"LockQueueLength", "Number of threads waiting to " +
       "acquire FSNameSystemLock"})
   public int getFsLockQueueLength() {
-    return fsLock.getQueueLength();
+    return fsLock.getQueueLength(FSNamesystemLockMode.FS);
   }
 
   @Metric(value = {"ReadLockLongHoldCount", "The number of time " +
           "the read lock has been held for longer than the threshold"},
           type = Metric.Type.COUNTER)
   public long getNumOfReadLockLongHold() {
-    return fsLock.getNumOfReadLockLongHold();
+    return fsLock.getNumOfReadLockLongHold(FSNamesystemLockMode.FS);
   }
 
   @Metric(value = {"WriteLockLongHoldCount", "The number of time " +
           "the write lock has been held for longer than the threshold"},
           type = Metric.Type.COUNTER)
   public long getNumOfWriteLockLongHold() {
-    return fsLock.getNumOfWriteLockLongHold();
+    return fsLock.getNumOfWriteLockLongHold(FSNamesystemLockMode.FS);
   }
 
   int getNumberOfDatanodes(DatanodeReportType type) {
@@ -7097,12 +7121,12 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   
   @VisibleForTesting
   void setFsLockForTests(ReentrantReadWriteLock lock) {
-    this.fsLock.coarseLock = lock;
+    this.fsLock.setLockForTests(lock);
   }
   
   @VisibleForTesting
   public ReentrantReadWriteLock getFsLockForTests() {
-    return fsLock.coarseLock;
+    return fsLock.getLockForTests();
   }
   
   @VisibleForTesting
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 af641e964f46..abdc80e1e81a 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
@@ -64,7 +64,7 @@ import static 
org.apache.hadoop.log.LogThrottlingHelper.LogAction;
  * {@link MutableRatesWithAggregation}. However since threads are re-used
  * between operations this should not generally be an issue.
  */
-class FSNamesystemLock {
+public class FSNamesystemLock {
   @VisibleForTesting
   protected ReentrantReadWriteLock coarseLock;
 
@@ -129,7 +129,7 @@ class FSNamesystemLock {
 
   private static final String OVERALL_METRIC_NAME = "Overall";
 
-  FSNamesystemLock(Configuration conf,
+  public FSNamesystemLock(Configuration conf,
       MutableRatesWithAggregation detailedHoldTimeMetrics) {
     this(conf, detailedHoldTimeMetrics, new Timer());
   }
@@ -489,6 +489,14 @@ class FSNamesystemLock {
     return writeLockReportingThresholdMs;
   }
 
+  public void setLockForTests(ReentrantReadWriteLock lock) {
+    this.coarseLock = lock;
+  }
+
+  public ReentrantReadWriteLock getLockForTests() {
+    return this.coarseLock;
+  }
+
   /**
    * Read lock Held Info.
    */
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FSNLockManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FSNLockManager.java
new file mode 100644
index 000000000000..8956a3461de5
--- /dev/null
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FSNLockManager.java
@@ -0,0 +1,184 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.fgl;
+
+import org.apache.hadoop.classification.VisibleForTesting;
+
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+
+public interface FSNLockManager {
+
+  /**
+   * Acquire read lock for an operation according to the lock mode.
+   * @param lockMode locking mode
+   */
+  void readLock(FSNamesystemLockMode lockMode);
+
+  /**
+   * Acquire read lock according to the lock mode, unless interrupted while 
waiting.
+   * @param lockMode locking mode
+   */
+  void readLockInterruptibly(FSNamesystemLockMode lockMode) throws 
InterruptedException;
+
+  /**
+   * Release read lock for the operation according to the lock mode.
+   * @param lockMode locking mode
+   * @param opName operation name
+   */
+  void readUnlock(FSNamesystemLockMode lockMode, String opName);
+
+  /**
+   * Release read lock for the operation according to the lock mode.
+   * @param lockMode locking mode
+   * @param opName operation name
+   * @param lockReportInfoSupplier supplier used to report some information 
for this lock.
+   */
+  void readUnlock(FSNamesystemLockMode lockMode, String opName,
+      Supplier<String> lockReportInfoSupplier);
+
+  /**
+   * Acquire write lock for an operation according to the lock mode.
+   * @param lockMode locking mode
+   */
+  void writeLock(FSNamesystemLockMode lockMode);
+
+  /**
+   * Release write lock for the operation according to the lock mode.
+   * @param lockMode locking mode
+   * @param opName operation name
+   */
+  void writeUnlock(FSNamesystemLockMode lockMode, String opName);
+
+  /**
+   * Release write lock for the operation according to the lock mode.
+   * @param lockMode locking mode
+   * @param opName operation name
+   * @param suppressWriteLockReport When false, event of write lock being held
+   * for long time will be logged in logs and metrics.
+   */
+  void writeUnlock(FSNamesystemLockMode lockMode, String opName,
+      boolean suppressWriteLockReport);
+
+  /**
+   * Release write lock for the operation according to the lock mode.
+   * @param lockMode locking mode
+   * @param opName operation name
+   * @param lockReportInfoSupplier supplier used to report information for 
this lock.
+   */
+  void writeUnlock(FSNamesystemLockMode lockMode, String opName,
+      Supplier<String> lockReportInfoSupplier);
+
+  void writeLockInterruptibly(FSNamesystemLockMode lockMode) throws 
InterruptedException;
+
+  /**
+   * Check if the current thread holds write lock according to the lock mode.
+   * @param lockMode locking mode
+   * @return true if the current thread is holding the write-lock, else false.
+   */
+  boolean hasWriteLock(FSNamesystemLockMode lockMode);
+
+  /**
+   * Check if the current thread holds read lock according to the lock mode.
+   * @param lockMode locking mode
+   * @return true if the current thread is holding the read-lock, else false.
+   */
+  boolean hasReadLock(FSNamesystemLockMode lockMode);
+
+  /**
+   * Queries the number of reentrant read holds on this lock by the
+   * current thread.  A reader thread has a hold on a lock for
+   * each lock action that is not matched by an unlock action.
+   *
+   * @param lockMode locking mode
+   * @return the number of holds on the read lock by the current thread,
+   *         or zero if the read lock is not held by the current thread
+   */
+  int getReadHoldCount(FSNamesystemLockMode lockMode);
+
+  /**
+   * Returns the QueueLength of waiting threads.
+   * A larger number indicates greater lock contention.
+   *
+   * @param lockMode locking mode
+   * @return int - Number of threads waiting on this lock
+   */
+  int getQueueLength(FSNamesystemLockMode lockMode);
+
+  /**
+   * Returns the number of time the read lock
+   * has been held longer than the threshold.
+   *
+   * @param lockMode locking mode
+   * @return long - Number of time the read lock
+   * has been held longer than the threshold
+   */
+  long getNumOfReadLockLongHold(FSNamesystemLockMode lockMode);
+
+  /**
+   * Returns the number of time the write-lock
+   * has been held longer than the threshold.
+   *
+   * @param lockMode locking mode
+   * @return long - Number of time the write-lock
+   * has been held longer than the threshold.
+   */
+  long getNumOfWriteLockLongHold(FSNamesystemLockMode lockMode);
+
+  /**
+   * Check if the metrics is enabled.
+   * @return true if the metrics is enabled, else false.
+   */
+  boolean isMetricsEnabled();
+
+  /**
+   * Reset the metricsEnabled according to the lock mode.
+   * @param metricsEnabled the new metricsEnabled
+   */
+  void setMetricsEnabled(boolean metricsEnabled);
+
+  /**
+   * Try to set the reporting threshold of the read lock.
+   * @param readLockReportingThresholdMs reporting threshold
+   */
+  void setReadLockReportingThresholdMs(long readLockReportingThresholdMs);
+
+  /**
+   * Try to get the reporting threshold of the read lock.
+   * @return the reporting threshold.
+   */
+  long getReadLockReportingThresholdMs();
+
+  /**
+   * Try to set the reporting threshold for the write lock.
+   * @param writeLockReportingThresholdMs reporting threshold.
+   */
+  void setWriteLockReportingThresholdMs(long writeLockReportingThresholdMs);
+
+  /**
+   * Try to get the reporting threshold for the write lock.
+   * @return reporting threshold.
+   */
+  long getWriteLockReportingThresholdMs();
+
+  @VisibleForTesting
+  void setLockForTests(ReentrantReadWriteLock lock);
+
+  @VisibleForTesting
+  ReentrantReadWriteLock getLockForTests();
+}
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FSNamesystemLockMode.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FSNamesystemLockMode.java
new file mode 100644
index 000000000000..f3b7d570db92
--- /dev/null
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FSNamesystemLockMode.java
@@ -0,0 +1,24 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.fgl;
+
+public enum FSNamesystemLockMode {
+  GLOBAL,
+  FS,
+  BM
+}
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/GlobalFSNamesystemLock.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/GlobalFSNamesystemLock.java
new file mode 100644
index 000000000000..c01529fac4c3
--- /dev/null
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/GlobalFSNamesystemLock.java
@@ -0,0 +1,149 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.fgl;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystemLock;
+import org.apache.hadoop.metrics2.lib.MutableRatesWithAggregation;
+
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import java.util.function.Supplier;
+
+public class GlobalFSNamesystemLock implements FSNLockManager {
+
+  private final FSNamesystemLock lock;
+
+  public GlobalFSNamesystemLock(Configuration conf, 
MutableRatesWithAggregation aggregation) {
+    this.lock = new FSNamesystemLock(conf, aggregation);
+  }
+
+  @Override
+  public void readLock(FSNamesystemLockMode lockMode) {
+    this.lock.readLock();
+  }
+
+  public void readLockInterruptibly(FSNamesystemLockMode lockMode) throws 
InterruptedException  {
+    this.lock.readLockInterruptibly();
+  }
+
+  @Override
+  public void readUnlock(FSNamesystemLockMode lockMode, String opName) {
+    this.lock.readUnlock(opName);
+  }
+
+  public void readUnlock(FSNamesystemLockMode lockMode, String opName,
+      Supplier<String> lockReportInfoSupplier) {
+    this.lock.readUnlock(opName, lockReportInfoSupplier);
+  }
+
+  @Override
+  public void writeLock(FSNamesystemLockMode lockMode) {
+    this.lock.writeLock();
+  }
+
+  @Override
+  public void writeUnlock(FSNamesystemLockMode lockMode, String opName) {
+    this.lock.writeUnlock(opName);
+  }
+
+  @Override
+  public void writeUnlock(FSNamesystemLockMode lockMode, String opName,
+      boolean suppressWriteLockReport) {
+    this.lock.writeUnlock(opName, suppressWriteLockReport);
+  }
+
+  public void writeUnlock(FSNamesystemLockMode lockMode, String opName,
+      Supplier<String> lockReportInfoSupplier) {
+    this.lock.writeUnlock(opName, lockReportInfoSupplier);
+  }
+
+  @Override
+  public void writeLockInterruptibly(FSNamesystemLockMode lockMode)
+      throws InterruptedException {
+    this.lock.writeLockInterruptibly();
+  }
+
+  @Override
+  public boolean hasWriteLock(FSNamesystemLockMode lockMode) {
+    return this.lock.isWriteLockedByCurrentThread();
+  }
+
+  @Override
+  public boolean hasReadLock(FSNamesystemLockMode lockMode) {
+    return this.lock.getReadHoldCount() > 0 || hasWriteLock(lockMode);
+  }
+
+  @Override
+  public int getReadHoldCount(FSNamesystemLockMode lockMode) {
+    return this.lock.getReadHoldCount();
+  }
+
+  @Override
+  public int getQueueLength(FSNamesystemLockMode lockMode) {
+    return this.lock.getQueueLength();
+  }
+
+  @Override
+  public long getNumOfReadLockLongHold(FSNamesystemLockMode lockMode) {
+    return this.lock.getNumOfReadLockLongHold();
+  }
+
+  @Override
+  public long getNumOfWriteLockLongHold(FSNamesystemLockMode lockMode) {
+    return this.lock.getNumOfWriteLockLongHold();
+  }
+
+  @Override
+  public boolean isMetricsEnabled() {
+    return this.lock.isMetricsEnabled();
+  }
+
+  public void setMetricsEnabled(boolean metricsEnabled) {
+    this.lock.setMetricsEnabled(metricsEnabled);
+  }
+
+  @Override
+  public void setReadLockReportingThresholdMs(long 
readLockReportingThresholdMs) {
+    this.lock.setReadLockReportingThresholdMs(readLockReportingThresholdMs);
+  }
+
+  @Override
+  public long getReadLockReportingThresholdMs() {
+    return this.lock.getReadLockReportingThresholdMs();
+  }
+
+  @Override
+  public void setWriteLockReportingThresholdMs(long 
writeLockReportingThresholdMs) {
+    this.lock.setWriteLockReportingThresholdMs(writeLockReportingThresholdMs);
+  }
+
+  @Override
+  public long getWriteLockReportingThresholdMs() {
+    return this.lock.getWriteLockReportingThresholdMs();
+  }
+
+  @Override
+  public void setLockForTests(ReentrantReadWriteLock testLock) {
+    this.lock.setLockForTests(testLock);
+  }
+
+  @Override
+  public ReentrantReadWriteLock getLockForTests() {
+    return this.lock.getLockForTests();
+  }
+}
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/package-info.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/package-info.java
new file mode 100644
index 000000000000..57ea1545910a
--- /dev/null
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/package-info.java
@@ -0,0 +1,18 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs.server.namenode.fgl;
\ No newline at end of file
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
index 05c1a06abda2..7398e2a91273 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java
@@ -17,41 +17,93 @@
  */
 package org.apache.hadoop.hdfs.util;
 
-/** Read-write lock interface. */
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
+
+/** Read-write lock interface for FSNamesystem. */
 public interface RwLock {
   /** Acquire read lock. */
-  public void readLock();
+  default void readLock() {
+    readLock(FSNamesystemLockMode.GLOBAL);
+  }
+
+  /** Acquire read lock. */
+  void readLock(FSNamesystemLockMode lockMode);
 
-  /** Acquire read lock, unless interrupted while waiting  */
-  void readLockInterruptibly() throws InterruptedException;
+  /** Acquire read lock, unless interrupted while waiting.  */
+  default void readLockInterruptibly() throws InterruptedException {
+    readLockInterruptibly(FSNamesystemLockMode.GLOBAL);
+  }
+
+  /** Acquire read lock, unless interrupted while waiting.  */
+  void readLockInterruptibly(FSNamesystemLockMode lockMode) throws 
InterruptedException;
 
   /** Release read lock. */
-  public void readUnlock();
+  default void readUnlock() {
+    readUnlock(FSNamesystemLockMode.GLOBAL, "OTHER");
+  }
 
   /**
    * Release read lock with operation name.
    * @param opName Option name.
    */
-  public void readUnlock(String opName);
+  default void readUnlock(String opName) {
+    readUnlock(FSNamesystemLockMode.GLOBAL, opName);
+  }
+
+  /**
+   * Release read lock with operation name.
+   * @param opName Option name.
+   */
+  void readUnlock(FSNamesystemLockMode lockMode, String opName);
+
+  /** Check if the current thread holds read lock. */
+  default boolean hasReadLock() {
+    return hasReadLock(FSNamesystemLockMode.GLOBAL);
+  }
 
   /** Check if the current thread holds read lock. */
-  public boolean hasReadLock();
+  boolean hasReadLock(FSNamesystemLockMode lockMode);
 
   /** Acquire write lock. */
-  public void writeLock();
+  default void writeLock() {
+    writeLock(FSNamesystemLockMode.GLOBAL);
+  }
+
+  /** Acquire write lock. */
+  void writeLock(FSNamesystemLockMode lockMode);
   
-  /** Acquire write lock, unless interrupted while waiting  */
-  void writeLockInterruptibly() throws InterruptedException;
+  /** Acquire write lock, unless interrupted while waiting.  */
+  default void writeLockInterruptibly() throws InterruptedException {
+    writeLockInterruptibly(FSNamesystemLockMode.GLOBAL);
+  }
+
+  /** Acquire write lock, unless interrupted while waiting.  */
+  void writeLockInterruptibly(FSNamesystemLockMode lockMode) throws 
InterruptedException;
 
   /** Release write lock. */
-  public void writeUnlock();
+  default void writeUnlock() {
+    writeUnlock(FSNamesystemLockMode.GLOBAL, "OTHER");
+  }
+
+  /**
+   * Release write lock with operation name.
+   * @param opName Option name.
+   */
+  default void writeUnlock(String opName) {
+    writeUnlock(FSNamesystemLockMode.GLOBAL, opName);
+  }
 
   /**
    * Release write lock with operation name.
    * @param opName Option name.
    */
-  public void writeUnlock(String opName);
+  void writeUnlock(FSNamesystemLockMode lockMode, String opName);
+
+  /** Check if the current thread holds write lock. */
+  default boolean hasWriteLock() {
+    return hasWriteLock(FSNamesystemLockMode.GLOBAL);
+  }
 
   /** Check if the current thread holds write lock. */
-  public boolean hasWriteLock();
+  boolean hasWriteLock(FSNamesystemLockMode lockMode);
 }
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
index e6dc8c5ba1ac..6fea6444c8fd 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml
@@ -3956,6 +3956,15 @@
   </description>
 </property>
 
+<property>
+  <name>dfs.namenode.lock.model.provider.class</name>
+  
<value>org.apache.hadoop.hdfs.server.namenode.fgl.GlobalFSNamesystemLock</value>
+  <description>
+    An implementation class of FSNamesystem lock.
+    Defaults to GlobalFSNamesystemLock.class
+  </description>
+</property>
+
 <property>
   <name>dfs.datanode.bp-ready.timeout</name>
   <value>20</value>


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

Reply via email to