Browse Source

HDFS-17697. [FGL] hasWriteLock and hasReadLock in FineGrainedFSNamesystemLock shouldn't throw assert error (#7250)

ZanderXu 5 tháng trước cách đây
mục cha
commit
8ab53982a5

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FSNLockManager.java

@@ -35,6 +35,7 @@ public interface FSNLockManager {
   /**
    * Acquire read lock according to the lock mode, unless interrupted while waiting.
    * @param lockMode locking mode
+   * @throws InterruptedException If the thread is interrupted, an InterruptedException is thrown.
    */
   void readLockInterruptibly(RwLockMode lockMode) throws InterruptedException;
 

+ 5 - 21
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/fgl/FineGrainedFSNamesystemLock.java

@@ -166,15 +166,8 @@ public class FineGrainedFSNamesystemLock implements FSNLockManager {
   @Override
   public boolean hasWriteLock(RwLockMode lockMode) {
     if (lockMode.equals(RwLockMode.GLOBAL)) {
-      if (this.fsLock.isWriteLockedByCurrentThread()) {
-        // The bm writeLock should be held by the current thread.
-        assert this.bmLock.isWriteLockedByCurrentThread();
-        return true;
-      } else {
-        // The bm writeLock should not be held by the current thread.
-        assert !this.bmLock.isWriteLockedByCurrentThread();
-        return false;
-      }
+      return this.fsLock.isWriteLockedByCurrentThread()
+          && this.bmLock.isWriteLockedByCurrentThread();
     } else if (lockMode.equals(RwLockMode.FS)) {
       return this.fsLock.isWriteLockedByCurrentThread();
     } else if (lockMode.equals(RwLockMode.BM)) {
@@ -186,17 +179,8 @@ public class FineGrainedFSNamesystemLock implements FSNLockManager {
   @Override
   public boolean hasReadLock(RwLockMode lockMode) {
     if (lockMode.equals(RwLockMode.GLOBAL)) {
-      if (hasWriteLock(RwLockMode.GLOBAL)) {
-        return true;
-      } else if (this.fsLock.getReadHoldCount() > 0) {
-        // The bm readLock should be held by the current thread.
-        assert this.bmLock.getReadHoldCount() > 0;
-        return true;
-      } else {
-        // The bm readLock should not be held by the current thread.
-        assert this.bmLock.getReadHoldCount() <= 0;
-        return false;
-      }
+      return hasWriteLock(RwLockMode.GLOBAL) ||
+          (this.fsLock.getReadHoldCount() > 0 && this.bmLock.getReadHoldCount() > 0);
     } else if (lockMode.equals(RwLockMode.FS)) {
       return this.fsLock.getReadHoldCount() > 0 || this.fsLock.isWriteLockedByCurrentThread();
     } else if (lockMode.equals(RwLockMode.BM)) {
@@ -205,11 +189,11 @@ public class FineGrainedFSNamesystemLock implements FSNLockManager {
     return false;
   }
 
-  @Override
   /**
    * This method is only used for ComputeDirectoryContentSummary.
    * For the GLOBAL mode, just return the FSLock's ReadHoldCount.
    */
+  @Override
   public int getReadHoldCount(RwLockMode lockMode) {
     if (lockMode.equals(RwLockMode.GLOBAL)) {
       return this.fsLock.getReadHoldCount();

+ 28 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/util/RwLock.java

@@ -24,7 +24,9 @@ public interface RwLock {
     readLock(RwLockMode.GLOBAL);
   }
 
-  /** Acquire read lock. */
+  /** Acquire read lock.
+   * @param lockMode The lock type for acquiring a read lock
+   */
   void readLock(RwLockMode lockMode);
 
   /** Acquire read lock, unless interrupted while waiting.  */
@@ -32,7 +34,9 @@ public interface RwLock {
     readLockInterruptibly(RwLockMode.GLOBAL);
   }
 
-  /** Acquire read lock, unless interrupted while waiting.  */
+  /** Acquire read lock, unless interrupted while waiting.
+   * @param lockMode The lock type for acquiring a read lock
+   */
   void readLockInterruptibly(RwLockMode lockMode) throws InterruptedException;
 
   /** Release read lock. */
@@ -50,16 +54,22 @@ public interface RwLock {
 
   /**
    * Release read lock with operation name.
+   * @param lockMode The lock type for releasing the read lock
    * @param opName Option name.
    */
   void readUnlock(RwLockMode lockMode, String opName);
 
-  /** Check if the current thread holds read lock. */
+  /** Check if the current thread holds read lock.
+   * @return true if the read lock is held by the current thread, else false
+   */
   default boolean hasReadLock() {
     return hasReadLock(RwLockMode.GLOBAL);
   }
 
-  /** Check if the current thread holds read lock. */
+  /** Check if the current thread holds read lock.
+   * @param lockMode The lock type used to check whether a read lock is held
+   * @return true if the read lock is held by the current thread, else false
+   */
   boolean hasReadLock(RwLockMode lockMode);
 
   /** Acquire write lock. */
@@ -67,7 +77,9 @@ public interface RwLock {
     writeLock(RwLockMode.GLOBAL);
   }
 
-  /** Acquire write lock. */
+  /** Acquire write lock.
+   * @param lockMode The lock type for acquiring a write lock
+   */
   void writeLock(RwLockMode lockMode);
   
   /** Acquire write lock, unless interrupted while waiting.  */
@@ -75,7 +87,9 @@ public interface RwLock {
     writeLockInterruptibly(RwLockMode.GLOBAL);
   }
 
-  /** Acquire write lock, unless interrupted while waiting.  */
+  /** Acquire write lock, unless interrupted while waiting.
+   * @param lockMode The lock type for acquiring a write lock
+   */
   void writeLockInterruptibly(RwLockMode lockMode) throws InterruptedException;
 
   /** Release write lock. */
@@ -93,15 +107,21 @@ public interface RwLock {
 
   /**
    * Release write lock with operation name.
+   * @param lockMode The lock type for releasing the write lock
    * @param opName Option name.
    */
   void writeUnlock(RwLockMode lockMode, String opName);
 
-  /** Check if the current thread holds write lock. */
+  /** Check if the current thread holds write lock.
+   * @return true if the write lock is held by the current thread, else false
+   */
   default boolean hasWriteLock() {
     return hasWriteLock(RwLockMode.GLOBAL);
   }
 
-  /** Check if the current thread holds write lock. */
+  /** Check if the current thread holds write lock.
+   * @param lockMode The lock type used to check whether a write lock is held
+   * @return true if the write lock is held by the current thread, else false.
+   */
   boolean hasWriteLock(RwLockMode lockMode);
 }