Przeglądaj źródła

HDFS-17410. [FGL] Client RPCs that changes file attributes supports fine-grained lock (#6634)

ZanderXu 1 rok temu
rodzic
commit
778e819b41

+ 5 - 4
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ContentSummaryComputationContext.java

@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -118,8 +119,8 @@ public class ContentSummaryComputationContext {
 
     boolean hadDirReadLock = dir.hasReadLock();
     boolean hadDirWriteLock = dir.hasWriteLock();
-    boolean hadFsnReadLock = fsn.hasReadLock();
-    boolean hadFsnWriteLock = fsn.hasWriteLock();
+    boolean hadFsnReadLock = fsn.hasReadLock(FSNamesystemLockMode.GLOBAL);
+    boolean hadFsnWriteLock = fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
 
     // sanity check.
     if (!hadDirReadLock || !hadFsnReadLock || hadDirWriteLock ||
@@ -130,14 +131,14 @@ public class ContentSummaryComputationContext {
 
     // unlock
     dir.readUnlock();
-    fsn.readUnlock("contentSummary");
+    fsn.readUnlock(FSNamesystemLockMode.GLOBAL, "contentSummary");
 
     try {
       Thread.sleep(sleepMilliSec, sleepNanoSec);
     } catch (InterruptedException ie) {
     } finally {
       // reacquire
-      fsn.readLock();
+      fsn.readLock(FSNamesystemLockMode.GLOBAL);
       dir.readLock();
     }
     yieldCount++;

+ 8 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/EncryptionZoneManager.java

@@ -48,6 +48,7 @@ import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot;
 import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.util.Lists;
@@ -188,11 +189,12 @@ public class EncryptionZoneManager {
       final int count) throws IOException {
     INodesInPath iip;
     final FSPermissionChecker pc = dir.getPermissionChecker();
-    dir.getFSNamesystem().readLock();
+    dir.getFSNamesystem().readLock(FSNamesystemLockMode.FS);
     try {
       iip = dir.resolvePath(pc, zone, DirOp.READ);
     } finally {
-      dir.getFSNamesystem().readUnlock("pauseForTestingAfterNthCheckpoint");
+      dir.getFSNamesystem().readUnlock(
+          FSNamesystemLockMode.FS, "pauseForTestingAfterNthCheckpoint");
     }
     reencryptionHandler
         .pauseForTestingAfterNthCheckpoint(iip.getLastINode().getId(), count);
@@ -213,7 +215,7 @@ public class EncryptionZoneManager {
       throws IOException {
     final FSPermissionChecker pc = dir.getPermissionChecker();
     final INode inode;
-    dir.getFSNamesystem().readLock();
+    dir.getFSNamesystem().readLock(FSNamesystemLockMode.FS);
     dir.readLock();
     try {
       final INodesInPath iip = dir.resolvePath(pc, zone, DirOp.READ);
@@ -224,7 +226,7 @@ public class EncryptionZoneManager {
       return getReencryptionStatus().getZoneStatus(inode.getId());
     } finally {
       dir.readUnlock();
-      dir.getFSNamesystem().readUnlock("getZoneStatus");
+      dir.getFSNamesystem().readUnlock(FSNamesystemLockMode.FS, "getZoneStatus");
     }
   }
 
@@ -281,11 +283,11 @@ public class EncryptionZoneManager {
     if (getProvider() == null || reencryptionHandler == null) {
       return;
     }
-    dir.getFSNamesystem().writeLock();
+    dir.getFSNamesystem().writeLock(FSNamesystemLockMode.FS);
     try {
       reencryptionHandler.stopThreads();
     } finally {
-      dir.getFSNamesystem().writeUnlock("stopReencryptThread");
+      dir.getFSNamesystem().writeUnlock(FSNamesystemLockMode.FS, "stopReencryptThread");
     }
     if (reencryptHandlerExecutor != null) {
       reencryptHandlerExecutor.shutdownNow();

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ZoneEncryptionInfoProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelperClient;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.ReencryptionUpdater.FileEdekInfo;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.Time;
@@ -382,7 +383,7 @@ final class FSDirEncryptionZoneOp {
    */
   static void saveFileXAttrsForBatch(FSDirectory fsd,
       List<FileEdekInfo> batch) {
-    assert fsd.getFSNamesystem().hasWriteLock();
+    assert fsd.getFSNamesystem().hasWriteLock(FSNamesystemLockMode.FS);
     if (batch != null && !batch.isEmpty()) {
       for (FileEdekInfo entry : batch) {
         final INode inode = fsd.getInode(entry.getInodeId());
@@ -727,13 +728,13 @@ final class FSDirEncryptionZoneOp {
       final FSPermissionChecker pc, final String zone) throws IOException {
     assert dir.getProvider() != null;
     final INodesInPath iip;
-    dir.getFSNamesystem().readLock();
+    dir.getFSNamesystem().readLock(FSNamesystemLockMode.FS);
     try {
       iip = dir.resolvePath(pc, zone, DirOp.READ);
       dir.ezManager.checkEncryptionZoneRoot(iip.getLastINode(), zone);
       return dir.ezManager.getKeyName(iip);
     } finally {
-      dir.getFSNamesystem().readUnlock("getKeyNameForZone");
+      dir.getFSNamesystem().readUnlock(FSNamesystemLockMode.FS, "getKeyNameForZone");
     }
   }
 }

+ 2 - 1
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSatisfyStoragePolicyOp.java

@@ -31,6 +31,7 @@ import org.apache.hadoop.fs.permission.FsAction;
 import org.apache.hadoop.hdfs.XAttrHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.hdfs.server.namenode.sps.StoragePolicySatisfyManager;
 import org.apache.hadoop.util.Lists;
 
@@ -64,7 +65,7 @@ final class FSDirSatisfyStoragePolicyOp {
   static FileStatus satisfyStoragePolicy(FSDirectory fsd, BlockManager bm,
       String src, boolean logRetryCache) throws IOException {
 
-    assert fsd.getFSNamesystem().hasWriteLock();
+    assert fsd.getFSNamesystem().hasWriteLock(FSNamesystemLockMode.FS);
     FSPermissionChecker pc = fsd.getPermissionChecker();
     INodesInPath iip;
     fsd.writeLock();

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 
 import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 
 /**
  * Helper class to perform truncate operation.
@@ -71,7 +72,7 @@ final class FSDirTruncateOp {
       final String clientMachine, final long mtime,
       final BlocksMapUpdateInfo toRemoveBlocks, final FSPermissionChecker pc)
       throws IOException, UnresolvedLinkException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
 
     FSDirectory fsd = fsn.getFSDirectory();
     final String src;
@@ -219,7 +220,7 @@ final class FSDirTruncateOp {
   static Block prepareFileForTruncate(FSNamesystem fsn, INodesInPath iip,
       String leaseHolder, String clientMachine, long lastBlockDelta,
       Block newBlock) throws IOException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
 
     INodeFile file = iip.getLastINode().asFile();
     assert !file.isStriped();
@@ -303,7 +304,7 @@ final class FSDirTruncateOp {
   private static boolean unprotectedTruncate(FSNamesystem fsn,
       INodesInPath iip, long newLength, BlocksMapUpdateInfo collectedBlocks,
       long mtime, QuotaCounts delta) throws IOException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
 
     INodeFile file = iip.getLastINode().asFile();
     int latestSnapshot = iip.getLatestSnapshotId();

+ 59 - 57
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -2156,13 +2156,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set permission for " + src);
         auditStat = FSDirAttrOp.setPermission(dir, pc, src, permission);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -2188,13 +2188,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set owner for " + src);
         auditStat = FSDirAttrOp.setOwner(dir, pc, src, username, group);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -2324,13 +2324,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     String srcsStr = Arrays.toString(srcs);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.GLOBAL);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot concat " + target);
         stat = FSDirConcatOp.concat(dir, pc, target, srcs, logRetryCache);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.GLOBAL, operationName,
             getLockReportInfoSupplier(srcsStr, target, stat));
       }
     } catch (AccessControlException ace) {
@@ -2353,13 +2353,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set times " + src);
         auditStat = FSDirAttrOp.setTimes(dir, pc, src, mtime, atime);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -2397,7 +2397,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       final FSPermissionChecker pc = getPermissionChecker();
       FSPermissionChecker.setOperationType(operationName);
-      writeLock();
+      writeLock(FSNamesystemLockMode.GLOBAL);
       BlocksMapUpdateInfo toRemoveBlocks = new BlocksMapUpdateInfo();
       try {
         checkOperation(OperationCategory.WRITE);
@@ -2406,7 +2406,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             clientMachine, mtime, toRemoveBlocks, pc);
       } finally {
         status = r != null ? r.getFileStatus() : null;
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.GLOBAL, operationName,
             getLockReportInfoSupplier(src, null, status));
       }
       getEditLog().logSync();
@@ -2477,14 +2477,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.GLOBAL);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set replication for " + src);
         success = FSDirAttrOp.setReplication(dir, pc, blockManager, src,
             replication);
       } finally {
-        writeUnlock(operationName, getLockReportInfoSupplier(src));
+        writeUnlock(FSNamesystemLockMode.GLOBAL, operationName, getLockReportInfoSupplier(src));
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
@@ -2537,14 +2537,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FSPermissionChecker.setOperationType(operationName);
     FileStatus auditStat = null;
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set storage policy for " + src);
         auditStat = FSDirAttrOp.setStoragePolicy(dir, pc, blockManager, src,
             policyName);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -2571,14 +2571,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     validateStoragePolicySatisfy();
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot satisfy storage policy for " + src);
         auditStat = FSDirSatisfyStoragePolicyOp.satisfyStoragePolicy(
             dir, blockManager, src, logRetryCache);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -2619,13 +2619,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FSPermissionChecker.setOperationType(operationName);
     FileStatus auditStat = null;
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot unset storage policy for " + src);
         auditStat = FSDirAttrOp.unsetStoragePolicy(dir, pc, blockManager, src);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -2648,12 +2648,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
-    readLock();
+    readLock(FSNamesystemLockMode.FS);
     try {
       checkOperation(OperationCategory.READ);
       return FSDirAttrOp.getStoragePolicy(dir, pc, blockManager, src);
     } finally {
-      readUnlock("getStoragePolicy");
+      readUnlock(FSNamesystemLockMode.FS, operationName);
     }
   }
 
@@ -2663,12 +2663,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   BlockStoragePolicy[] getStoragePolicies() throws IOException {
     checkOperation(OperationCategory.READ);
-    readLock();
+    readLock(FSNamesystemLockMode.BM);
     try {
       checkOperation(OperationCategory.READ);
       return FSDirAttrOp.getStoragePolicies(blockManager);
     } finally {
-      readUnlock("getStoragePolicies");
+      readUnlock(FSNamesystemLockMode.BM, "getStoragePolicies");
     }
   }
 
@@ -2935,7 +2935,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   boolean recoverLeaseInternal(RecoverLeaseOp op, INodesInPath iip,
       String src, String holder, String clientMachine, boolean force)
       throws IOException {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     INodeFile file = iip.getLastINode().asFile();
     if (file.isUnderConstruction()) {
       //
@@ -3280,7 +3280,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @param blockType is the file under striping or contiguous layout?
    */
   Block createNewBlock(BlockType blockType) throws IOException {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     Block b = new Block(nextBlockId(blockType), 0, 0);
     // Increment the generation stamp for every new block.
     b.setGenerationStamp(nextGenerationStamp(false));
@@ -3633,12 +3633,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      readLock();
+      readLock(FSNamesystemLockMode.GLOBAL);
       try {
         checkOperation(OperationCategory.READ);
         quotaUsage = FSDirStatAndListingOp.getQuotaUsage(dir, pc, src);
       } finally {
-        readUnlock(operationName, getLockReportInfoSupplier(src));
+        readUnlock(FSNamesystemLockMode.GLOBAL, operationName, getLockReportInfoSupplier(src));
       }
     } catch (AccessControlException ace) {
       logAuditEvent(false, operationName, src);
@@ -3671,14 +3671,15 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       checkSuperuserPrivilege(operationName, src);
     }
     try {
-      writeLock();
+      // Need to compute the curren space usage
+      writeLock(FSNamesystemLockMode.GLOBAL);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set quota on " + src);
         FSDirAttrOp.setQuota(dir, pc, src, nsQuota, ssQuota, type,
             allowOwnerSetQuota);
       } finally {
-        writeUnlock(operationName, getLockReportInfoSupplier(src));
+        writeUnlock(FSNamesystemLockMode.GLOBAL, operationName, getLockReportInfoSupplier(src));
       }
     } catch (AccessControlException ace) {
       logAuditEvent(false, operationName, src);
@@ -5882,7 +5883,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   long nextGenerationStamp(boolean legacyBlock)
       throws IOException {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     checkNameNodeSafeMode("Cannot get next generation stamp");
 
     long gs = blockManager.nextGenerationStamp(legacyBlock);
@@ -8019,13 +8020,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
         auditStat = FSDirAclOp.modifyAclEntries(dir, pc, src, aclSpec);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -8044,13 +8045,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
         auditStat = FSDirAclOp.removeAclEntries(dir, pc, src, aclSpec);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -8068,13 +8069,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
         auditStat = FSDirAclOp.removeDefaultAcl(dir, pc, src);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -8092,13 +8093,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot remove ACL on " + src);
         auditStat = FSDirAclOp.removeAcl(dir, pc, src);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -8116,13 +8117,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set ACL on " + src);
         auditStat = FSDirAclOp.setAcl(dir, pc, src, aclSpec);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -8140,12 +8141,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      readLock();
+      readLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.READ);
         ret = FSDirAclOp.getAclStatus(dir, pc, src);
       } finally {
-        readUnlock(operationName, getLockReportInfoSupplier(src));
+        readUnlock(FSNamesystemLockMode.FS, operationName, getLockReportInfoSupplier(src));
       }
     } catch(AccessControlException ace) {
       logAuditEvent(false, operationName, src);
@@ -8212,7 +8213,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FSPermissionChecker.setOperationType(operationName);
     checkOperation(OperationCategory.READ);
     try {
-      readLock();
+      readLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.READ);
         Entry<EncryptionZone, FileStatus> ezForPath = FSDirEncryptionZoneOp
@@ -8220,7 +8221,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         resultingStat = ezForPath.getValue();
         encryptionZone = ezForPath.getKey();
       } finally {
-        readUnlock(operationName,
+        readUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(srcArg, null, resultingStat));
       }
     } catch (AccessControlException ace) {
@@ -8237,7 +8238,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = false;
     checkOperation(OperationCategory.READ);
     checkSuperuserPrivilege(operationName, dir.rootDir.getFullPathName());
-    readLock();
+    readLock(FSNamesystemLockMode.FS);
     try {
       checkOperation(OperationCategory.READ);
       final BatchedListEntries<EncryptionZone> ret =
@@ -8245,7 +8246,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       success = true;
       return ret;
     } finally {
-      readUnlock(operationName, getLockReportInfoSupplier(null));
+      readUnlock(FSNamesystemLockMode.FS, operationName, getLockReportInfoSupplier(null));
       logAuditEvent(success, operationName, null);
     }
   }
@@ -8274,7 +8275,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = false;
     checkOperation(OperationCategory.READ);
     checkSuperuserPrivilege(operationName, dir.rootDir.getFullPathName());
-    readLock();
+    readLock(FSNamesystemLockMode.FS);
     try {
       checkOperation(OperationCategory.READ);
       final BatchedListEntries<ZoneReencryptionStatus> ret =
@@ -8282,7 +8283,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       success = true;
       return ret;
     } finally {
-      readUnlock(operationName, getLockReportInfoSupplier(null));
+      readUnlock(FSNamesystemLockMode.FS, operationName, getLockReportInfoSupplier(null));
       logAuditEvent(success, operationName, null);
     }
   }
@@ -8305,7 +8306,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       LOG.info("Re-encryption using key version " + keyVersionName
           + " for zone " + zone);
     }
-    writeLock();
+    writeLock(FSNamesystemLockMode.FS);
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("NameNode in safemode, cannot " + action
@@ -8337,7 +8338,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         getEditLog().logSetXAttrs(zone, xattrs, logRetryCache);
       }
     } finally {
-      writeUnlock(action + "reencryption", getLockReportInfoSupplier(zone));
+      writeUnlock(FSNamesystemLockMode.FS,
+          action + "reencryption", getLockReportInfoSupplier(zone));
     }
     getEditLog().logSync();
   }
@@ -8663,14 +8665,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot set XAttr on " + src);
         auditStat = FSDirXAttrOp.setXAttr(dir, pc, src, xAttr, flag,
             logRetryCache);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -8689,12 +8691,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      readLock();
+      readLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.READ);
         fsXattrs = FSDirXAttrOp.getXAttrs(dir, pc, src, xAttrs);
       } finally {
-        readUnlock(operationName, getLockReportInfoSupplier(src));
+        readUnlock(FSNamesystemLockMode.FS, operationName, getLockReportInfoSupplier(src));
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
@@ -8711,12 +8713,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      readLock();
+      readLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.READ);
         fsXattrs = FSDirXAttrOp.listXAttrs(dir, pc, src);
       } finally {
-        readUnlock(operationName, getLockReportInfoSupplier(src));
+        readUnlock(FSNamesystemLockMode.FS, operationName, getLockReportInfoSupplier(src));
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
@@ -8734,14 +8736,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
         auditStat = FSDirXAttrOp.removeXAttr(dir, pc, src, xAttr,
             logRetryCache);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {

+ 6 - 5
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionHandler.java

@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.VisibleForTesting;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.util.Preconditions;
 import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -338,7 +339,7 @@ public class ReencryptionHandler implements Runnable {
       }
 
       final Long zoneId;
-      dir.getFSNamesystem().readLock();
+      dir.getFSNamesystem().readLock(FSNamesystemLockMode.FS);
       try {
         zoneId = getReencryptionStatus().getNextUnprocessedZone();
         if (zoneId == null) {
@@ -350,7 +351,7 @@ public class ReencryptionHandler implements Runnable {
         getReencryptionStatus().markZoneStarted(zoneId);
         resetSubmissionTracker(zoneId);
       } finally {
-        dir.getFSNamesystem().readUnlock("reEncryptThread");
+        dir.getFSNamesystem().readUnlock(FSNamesystemLockMode.FS, "reEncryptThread");
       }
 
       try {
@@ -442,7 +443,7 @@ public class ReencryptionHandler implements Runnable {
 
   List<XAttr> completeReencryption(final INode zoneNode) throws IOException {
     assert dir.hasWriteLock();
-    assert dir.getFSNamesystem().hasWriteLock();
+    assert dir.getFSNamesystem().hasWriteLock(FSNamesystemLockMode.FS);
     final Long zoneId = zoneNode.getId();
     ZoneReencryptionStatus zs = getReencryptionStatus().getZoneStatus(zoneId);
     assert zs != null;
@@ -613,7 +614,7 @@ public class ReencryptionHandler implements Runnable {
     protected void checkPauseForTesting()
         throws InterruptedException {
       assert !dir.hasReadLock();
-      assert !dir.getFSNamesystem().hasReadLock();
+      assert !dir.getFSNamesystem().hasReadLock(FSNamesystemLockMode.FS);
       while (shouldPauseForTesting) {
         LOG.info("Sleeping in the re-encrypt handler for unit test.");
         synchronized (reencryptionHandler) {
@@ -747,7 +748,7 @@ public class ReencryptionHandler implements Runnable {
     @Override
     protected void throttle() throws InterruptedException {
       assert !dir.hasReadLock();
-      assert !dir.getFSNamesystem().hasReadLock();
+      assert !dir.getFSNamesystem().hasReadLock(FSNamesystemLockMode.FS);
       final int numCores = Runtime.getRuntime().availableProcessors();
       if (taskQueue.size() >= numCores) {
         LOG.debug("Re-encryption handler throttling because queue size {} is"

+ 4 - 3
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ReencryptionUpdater.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.ZoneReencryptionStatus;
 import org.apache.hadoop.hdfs.server.namenode.ReencryptionHandler.ReencryptionBatch;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.util.Lists;
 import org.apache.hadoop.util.StopWatch;
@@ -434,7 +435,7 @@ public final class ReencryptionUpdater implements Runnable {
 
     boolean shouldRetry;
     do {
-      dir.getFSNamesystem().writeLock();
+      dir.getFSNamesystem().writeLock(FSNamesystemLockMode.FS);
       try {
         throttleTimerLocked.start();
         processTask(task);
@@ -452,7 +453,7 @@ public final class ReencryptionUpdater implements Runnable {
         task.processed = true;
         shouldRetry = false;
       } finally {
-        dir.getFSNamesystem().writeUnlock("reencryptUpdater");
+        dir.getFSNamesystem().writeUnlock(FSNamesystemLockMode.FS, "reencryptUpdater");
         throttleTimerLocked.stop();
       }
       // logSync regardless, to prevent edit log buffer overflow triggering
@@ -500,7 +501,7 @@ public final class ReencryptionUpdater implements Runnable {
 
   private synchronized void checkPauseForTesting() throws InterruptedException {
     assert !dir.hasWriteLock();
-    assert !dir.getFSNamesystem().hasWriteLock();
+    assert !dir.getFSNamesystem().hasWriteLock(FSNamesystemLockMode.FS);
     if (pauseAfterNthCheckpoint != 0) {
       ZoneSubmissionTracker tracker =
           handler.unprotectedGetTracker(pauseZoneId);