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 f0da5122e6d86f0410aee49da1eab5a95ac182cc
Author: ZanderXu <zande...@apache.org>
AuthorDate: Fri Mar 29 18:16:07 2024 +0800

    HDFS-17388. [FGL] Client RPCs involving write process supports fine-grained 
lock (#6589)
---
 .../hdfs/server/blockmanagement/BlockManager.java  |  2 +-
 .../hadoop/hdfs/server/namenode/FSDirAppendOp.java |  5 +-
 .../hadoop/hdfs/server/namenode/FSDirDeleteOp.java |  4 +-
 .../server/namenode/FSDirEncryptionZoneOp.java     |  8 +-
 .../server/namenode/FSDirStatAndListingOp.java     | 25 ++++--
 .../hdfs/server/namenode/FSDirWriteFileOp.java     | 43 +++++-----
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  | 96 +++++++++++-----------
 .../hadoop/hdfs/server/namenode/INodeFile.java     |  6 ++
 8 files changed, 107 insertions(+), 82 deletions(-)

diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index cf3ec7964c06..324a89214599 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1579,7 +1579,7 @@ public class BlockManager implements BlockStatsMXBean {
       final boolean inSnapshot, FileEncryptionInfo feInfo,
       ErasureCodingPolicy ecPolicy)
       throws IOException {
-    assert namesystem.hasReadLock();
+    assert namesystem.hasReadLock(FSNamesystemLockMode.BM);
     if (blocks == null) {
       return null;
     } else if (blocks.length == 0) {
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
index ba00e8ae936a..058928c41abb 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java
@@ -38,6 +38,7 @@ import 
org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem.RecoverLeaseOp;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeLayoutVersion.Feature;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.ipc.RetriableException;
 
 import org.apache.hadoop.util.Preconditions;
@@ -82,7 +83,7 @@ final class FSDirAppendOp {
       final String srcArg, final FSPermissionChecker pc, final String holder,
       final String clientMachine, final boolean newBlock,
       final boolean logRetryCache) throws IOException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
 
     final LocatedBlock lb;
     final FSDirectory fsd = fsn.getFSDirectory();
@@ -180,7 +181,7 @@ final class FSDirAppendOp {
       final String clientMachine, final boolean newBlock,
       final boolean writeToEditLog, final boolean logRetryCache)
       throws IOException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
 
     final INodeFile file = iip.getLastINode().asFile();
     final QuotaCounts delta = verifyQuotaForUCBlock(fsn, file, iip);
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
index 2dfb90ee6728..f513357d9c53 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirDeleteOp.java
@@ -24,6 +24,7 @@ import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory.DirOp;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 import org.apache.hadoop.hdfs.server.namenode.INode.ReclaimContext;
+import org.apache.hadoop.hdfs.server.namenode.fgl.FSNamesystemLockMode;
 import org.apache.hadoop.util.ChunkedArrayList;
 
 import java.io.IOException;
@@ -170,7 +171,8 @@ class FSDirDeleteOp {
   static BlocksMapUpdateInfo deleteInternal(
       FSNamesystem fsn, INodesInPath iip, boolean logRetryCache)
       throws IOException {
-    assert fsn.hasWriteLock();
+    // Delete INode and modify BlockInfo
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
     if (NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.delete: " + 
iip.getPath());
     }
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
index 9520139577ba..0ce2ce75151b 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirEncryptionZoneOp.java
@@ -84,8 +84,8 @@ final class FSDirEncryptionZoneOp {
   private static EncryptedKeyVersion generateEncryptedDataEncryptionKey(
       final FSDirectory fsd, final String ezKeyName) throws IOException {
     // must not be holding lock during this operation
-    assert !fsd.getFSNamesystem().hasReadLock();
-    assert !fsd.getFSNamesystem().hasWriteLock();
+    assert !fsd.getFSNamesystem().hasReadLock(FSNamesystemLockMode.FS);
+    assert !fsd.getFSNamesystem().hasWriteLock(FSNamesystemLockMode.FS);
     if (ezKeyName == null) {
       return null;
     }
@@ -657,13 +657,13 @@ final class FSDirEncryptionZoneOp {
     Preconditions.checkNotNull(ezKeyName);
 
     // Generate EDEK while not holding the fsn lock.
-    fsn.writeUnlock("getEncryptionKeyInfo");
+    fsn.writeUnlock(FSNamesystemLockMode.FS, "getEncryptionKeyInfo");
     try {
       EncryptionFaultInjector.getInstance().startFileBeforeGenerateKey();
       return new EncryptionKeyInfo(protocolVersion, suite, ezKeyName,
           generateEncryptedDataEncryptionKey(fsd, ezKeyName));
     } finally {
-      fsn.writeLock();
+      fsn.writeLock(FSNamesystemLockMode.FS);
       EncryptionFaultInjector.getInstance().startFileAfterGenerateKey();
     }
   }
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 4547228364d5..2fe8c307d2cd 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -18,6 +18,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.fs.ContentSummary;
@@ -444,16 +445,22 @@ class FSDirStatAndListingOp {
       if (isEncrypted) {
         feInfo = FSDirEncryptionZoneOp.getFileEncryptionInfo(fsd, iip);
       }
+      // ComputeFileSize and needLocation need BM lock.
       if (needLocation) {
-        final boolean inSnapshot = snapshot != Snapshot.CURRENT_STATE_ID;
-        final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
-        final long fileSize = !inSnapshot && isUc
-            ? fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
-        loc = fsd.getBlockManager().createLocatedBlocks(
-            fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size,
-            needBlockToken, inSnapshot, feInfo, ecPolicy);
-        if (loc == null) {
-          loc = new LocatedBlocks();
+        fsd.getFSNamesystem().readLock(FSNamesystemLockMode.BM);
+        try {
+          final boolean inSnapshot = snapshot != Snapshot.CURRENT_STATE_ID;
+          final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
+          final long fileSize = !inSnapshot && isUc
+              ? fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
+          loc = fsd.getBlockManager().createLocatedBlocks(
+              fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size,
+              needBlockToken, inSnapshot, feInfo, ecPolicy);
+          if (loc == null) {
+            loc = new LocatedBlocks();
+          }
+        } finally {
+          fsd.getFSNamesystem().readUnlock(FSNamesystemLockMode.BM, 
"createFileStatus");
         }
       }
     } else if (node.isDirectory()) {
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
index 37000421abcb..3067162e3a1e 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java
@@ -364,7 +364,7 @@ class FSDirWriteFileOp {
       boolean shouldReplicate, String ecPolicyName, String storagePolicy,
       boolean logRetryEntry)
       throws IOException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.FS);
     boolean overwrite = flag.contains(CreateFlag.OVERWRITE);
     boolean isLazyPersist = flag.contains(CreateFlag.LAZY_PERSIST);
 
@@ -372,22 +372,27 @@ class FSDirWriteFileOp {
     FSDirectory fsd = fsn.getFSDirectory();
 
     if (iip.getLastINode() != null) {
-      if (overwrite) {
-        List<INode> toRemoveINodes = new ChunkedArrayList<>();
-        List<Long> toRemoveUCFiles = new ChunkedArrayList<>();
-        long ret = FSDirDeleteOp.delete(fsd, iip, toRemoveBlocks,
-                                        toRemoveINodes, toRemoveUCFiles, 
now());
-        if (ret >= 0) {
-          iip = INodesInPath.replace(iip, iip.length() - 1, null);
-          FSDirDeleteOp.incrDeletedFileCount(ret);
-          fsn.removeLeasesAndINodes(toRemoveUCFiles, toRemoveINodes, true);
+      fsn.writeLock(FSNamesystemLockMode.BM);
+      try {
+        if (overwrite) {
+          List<INode> toRemoveINodes = new ChunkedArrayList<>();
+          List<Long> toRemoveUCFiles = new ChunkedArrayList<>();
+          long ret = FSDirDeleteOp.delete(fsd, iip, toRemoveBlocks,
+              toRemoveINodes, toRemoveUCFiles, now());
+          if (ret >= 0) {
+            iip = INodesInPath.replace(iip, iip.length() - 1, null);
+            FSDirDeleteOp.incrDeletedFileCount(ret);
+            fsn.removeLeasesAndINodes(toRemoveUCFiles, toRemoveINodes, true);
+          }
+        } else {
+          // If lease soft limit time is expired, recover the lease
+          fsn.recoverLeaseInternal(FSNamesystem.RecoverLeaseOp.CREATE_FILE, 
iip,
+              src, holder, clientMachine, false);
+          throw new FileAlreadyExistsException(src + " for client " +
+              clientMachine + " already exists");
         }
-      } else {
-        // If lease soft limit time is expired, recover the lease
-        fsn.recoverLeaseInternal(FSNamesystem.RecoverLeaseOp.CREATE_FILE, iip,
-                                 src, holder, clientMachine, false);
-        throw new FileAlreadyExistsException(src + " for client " +
-            clientMachine + " already exists");
+      } finally {
+        fsn.writeUnlock(FSNamesystemLockMode.BM, "create");
       }
     }
     fsn.checkFsObjectLimit();
@@ -597,7 +602,7 @@ class FSDirWriteFileOp {
       FSNamesystem fsn, INodesInPath iip, long fileId, String clientName,
       ExtendedBlock previous, LocatedBlock[] onRetryBlock)
       throws IOException {
-    assert fsn.hasReadLock();
+    assert fsn.hasReadLock(FSNamesystemLockMode.GLOBAL);
     String src = iip.getPath();
     checkBlock(fsn, previous);
     onRetryBlock[0] = null;
@@ -695,7 +700,7 @@ class FSDirWriteFileOp {
       FSNamesystem fsn, INodesInPath iip,
       String holder, Block last, long fileId)
       throws IOException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
     final String src = iip.getPath();
     final INodeFile pendingFile;
     INode inode = null;
@@ -779,7 +784,7 @@ class FSDirWriteFileOp {
   static void saveAllocatedBlock(FSNamesystem fsn, String src,
       INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets,
       BlockType blockType) throws IOException {
-    assert fsn.hasWriteLock();
+    assert fsn.hasWriteLock(FSNamesystemLockMode.GLOBAL);
     BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets,
         blockType);
     logAllocatedBlock(src, b);
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 fb983974458b..8eefe98383ea 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
@@ -2433,14 +2433,14 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot create symlink " + link);
         auditStat = FSDirSymlinkOp.createSymlinkInt(this, target, link,
             dirPerms, createParent, logRetryCache);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(link, target, auditStat));
       }
     } catch (AccessControlException e) {
@@ -2787,7 +2787,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(null);
-    writeLock();
+    writeLock(FSNamesystemLockMode.FS);
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot create file" + src);
@@ -2849,7 +2849,8 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
         dir.writeUnlock();
       }
     } finally {
-      writeUnlock("create", getLockReportInfoSupplier(src, null, stat));
+      writeUnlock(FSNamesystemLockMode.FS, "create",
+          getLockReportInfoSupplier(src, null, stat));
       // There might be transactions logged while trying to recover the lease.
       // They need to be sync'ed even when an exception was thrown.
       if (!skipSync) {
@@ -2882,7 +2883,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(null);
-    writeLock();
+    writeLock(FSNamesystemLockMode.GLOBAL);
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot recover the lease of " + src);
@@ -2902,7 +2903,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       skipSync = true;
       throw se;
     } finally {
-      writeUnlock("recoverLease");
+      writeUnlock(FSNamesystemLockMode.GLOBAL, "recoverLease");
       // There might be transactions logged while trying to recover the lease.
       // They need to be sync'ed even when an exception was thrown.
       if (!skipSync) {
@@ -3022,7 +3023,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       checkOperation(OperationCategory.WRITE);
       final FSPermissionChecker pc = getPermissionChecker();
       FSPermissionChecker.setOperationType(operationName);
-      writeLock();
+      writeLock(FSNamesystemLockMode.GLOBAL);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot append to file" + srcArg);
@@ -3032,7 +3033,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
         skipSync = true;
         throw se;
       } finally {
-        writeUnlock(operationName, getLockReportInfoSupplier(srcArg));
+        writeUnlock(FSNamesystemLockMode.GLOBAL, operationName, 
getLockReportInfoSupplier(srcArg));
         // There might be transactions logged while trying to recover the lease
         // They need to be sync'ed even when an exception was thrown.
         if (!skipSync) {
@@ -3079,13 +3080,13 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
-    readLock();
+    readLock(FSNamesystemLockMode.GLOBAL);
     try {
       checkOperation(OperationCategory.WRITE);
       r = FSDirWriteFileOp.validateAddBlock(this, pc, src, fileId, clientName,
                                             previous, onRetryBlock);
     } finally {
-      readUnlock(operationName);
+      readUnlock(FSNamesystemLockMode.GLOBAL, operationName);
     }
 
     if (r == null) {
@@ -3098,14 +3099,14 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
         blockManager, src, excludedNodes, favoredNodes, flags, r);
 
     checkOperation(OperationCategory.WRITE);
-    writeLock();
+    writeLock(FSNamesystemLockMode.GLOBAL);
     LocatedBlock lb;
     try {
       checkOperation(OperationCategory.WRITE);
       lb = FSDirWriteFileOp.storeAllocatedBlock(
           this, src, fileId, clientName, previous, targets);
     } finally {
-      writeUnlock(operationName);
+      writeUnlock(FSNamesystemLockMode.GLOBAL, operationName);
     }
     getEditLog().logSync();
     return lb;
@@ -3130,7 +3131,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(null);
-    readLock();
+    readLock(FSNamesystemLockMode.FS);
     try {
       // Changing this operation category to WRITE instead of making 
getAdditionalDatanode as a
       // read method is aim to let Active NameNode to handle this RPC, because 
Active NameNode
@@ -3155,7 +3156,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
           "src=%s, fileId=%d, blk=%s, clientName=%s, clientMachine=%s",
           src, fileId, blk, clientName, clientMachine));
     } finally {
-      readUnlock("getAdditionalDatanode");
+      readUnlock(FSNamesystemLockMode.FS, "getAdditionalDatanode");
     }
 
     if (clientnode == null) {
@@ -3181,7 +3182,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(null);
-    writeLock();
+    writeLock(FSNamesystemLockMode.GLOBAL);
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot abandon block " + b + " for file" + src);
@@ -3189,7 +3190,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       NameNode.stateChangeLog.debug(
           "BLOCK* NameSystem.abandonBlock: {} is removed from pendingCreates", 
b);
     } finally {
-      writeUnlock("abandonBlock");
+      writeUnlock(FSNamesystemLockMode.GLOBAL, "abandonBlock");
     }
     getEditLog().logSync();
   }
@@ -3204,7 +3205,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       throws LeaseExpiredException, FileNotFoundException {
     String src = iip.getPath();
     INode inode = iip.getLastINode();
-    assert hasReadLock();
+    assert hasReadLock(FSNamesystemLockMode.FS);
     if (inode == null) {
       throw new FileNotFoundException("File does not exist: "
           + leaseExceptionString(src, fileId, holder));
@@ -3247,14 +3248,14 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(null);
-    writeLock();
+    writeLock(FSNamesystemLockMode.GLOBAL);
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot complete file " + src);
       success = FSDirWriteFileOp.completeFile(this, pc, src, holder, last,
                                               fileId);
     } finally {
-      writeUnlock("completeFile");
+      writeUnlock(FSNamesystemLockMode.GLOBAL, "completeFile");
     }
     getEditLog().logSync();
     if (success) {
@@ -3269,6 +3270,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
    * @param blockType is the file under striping or contiguous layout?
    */
   Block createNewBlock(BlockType blockType) throws IOException {
+    // nextBlockId and nextGenerationStamp need to write edit log, so it needs 
FSLock.
     assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     Block b = new Block(nextBlockId(blockType), 0, 0);
     // Increment the generation stamp for every new block.
@@ -3282,7 +3284,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
    * all blocks, otherwise check only penultimate block.
    */
   boolean checkFileProgress(String src, INodeFile v, boolean checkall) {
-    assert hasReadLock();
+    assert hasReadLock(FSNamesystemLockMode.GLOBAL);
     if (checkall) {
       return checkBlocksComplete(src, true, v.getBlocks());
     } else {
@@ -3328,14 +3330,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 rename " + src);
         ret = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache);
       } finally {
         FileStatus status = ret != null ? ret.auditStat : null;
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, dst, status));
       }
     } catch (AccessControlException e)  {
@@ -3360,7 +3362,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(operationName);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.GLOBAL);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot rename " + src);
@@ -3368,7 +3370,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
             options);
       } finally {
         FileStatus status = res != null ? res.auditStat : null;
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.GLOBAL, operationName,
             getLockReportInfoSupplier(src, dst, status));
       }
     } catch (AccessControlException e) {
@@ -3403,7 +3405,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     FSPermissionChecker.setOperationType(operationName);
     boolean ret = false;
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.GLOBAL);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot delete " + src);
@@ -3411,7 +3413,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
             this, pc, src, recursive, logRetryCache);
         ret = toRemovedBlocks != null;
       } finally {
-        writeUnlock(operationName, getLockReportInfoSupplier(src));
+        writeUnlock(FSNamesystemLockMode.GLOBAL, operationName, 
getLockReportInfoSupplier(src));
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
@@ -3441,7 +3443,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   void removeLeasesAndINodes(List<Long> removedUCFiles,
       List<INode> removedINodes,
       final boolean acquireINodeMapLock) {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.FS);
     for(long i : removedUCFiles) {
       leaseManager.removeLease(i);
     }
@@ -3546,14 +3548,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 create directory " + src);
         auditStat = FSDirMkdirOp.mkdirs(this, pc, src, permissions,
             createParent);
       } finally {
-        writeUnlock(operationName,
+        writeUnlock(FSNamesystemLockMode.FS, operationName,
             getLockReportInfoSupplier(src, null, auditStat));
       }
     } catch (AccessControlException e) {
@@ -3693,7 +3695,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
     FSPermissionChecker.setOperationType(null);
-    writeLock();
+    writeLock(FSNamesystemLockMode.GLOBAL);
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot fsync file " + src);
@@ -3706,7 +3708,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       }
       FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, false);
     } finally {
-      writeUnlock("fsync");
+      writeUnlock(FSNamesystemLockMode.GLOBAL, "fsync");
     }
     getEditLog().logSync();
   }
@@ -3729,7 +3731,8 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       String recoveryLeaseHolder) throws IOException {
     LOG.info("Recovering " + lease + ", src=" + src);
     assert !isInSafeMode();
-    assert hasWriteLock();
+    // finalizeINodeFileUnderConstruction needs global write lock.
+    assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
 
     final INodeFile pendingFile = iip.getLastINode().asFile();
     int nrBlocks = pendingFile.numBlocks();
@@ -3891,7 +3894,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
 
   private Lease reassignLease(Lease lease, String src, String newHolder,
       INodeFile pendingFile) {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.FS);
     if(newHolder == null)
       return lease;
     // The following transaction is not synced. Make sure it's sync'ed later.
@@ -3900,7 +3903,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   }
   
   Lease reassignLeaseInternal(Lease lease, String newHolder, INodeFile 
pendingFile) {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.FS);
     pendingFile.getFileUnderConstructionFeature().setClientName(newHolder);
     return leaseManager.reassignLease(lease, pendingFile, newHolder);
   }
@@ -5861,6 +5864,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
    */
   long nextGenerationStamp(boolean legacyBlock)
       throws IOException {
+    // TODO: Use FSLock to make nextGenerationStamp thread safe.
     assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     checkNameNodeSafeMode("Cannot get next generation stamp");
 
@@ -5880,7 +5884,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
    * @param blockType is the file under striping or contiguous layout?
    */
   private long nextBlockId(BlockType blockType) throws IOException {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     checkNameNodeSafeMode("Cannot get next block ID");
     final long blockId = blockManager.nextBlockId(blockType);
     getEditLog().logAllocateBlockId(blockId);
@@ -5929,7 +5933,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
 
   private INodeFile checkUCBlock(ExtendedBlock block,
       String clientName) throws IOException {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     checkNameNodeSafeMode("Cannot get a new generation stamp and an "
         + "access token for block " + block);
     
@@ -6003,7 +6007,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       String clientName) throws IOException {
     final LocatedBlock locatedBlock;
     checkOperation(OperationCategory.WRITE);
-    writeLock();
+    writeLock(FSNamesystemLockMode.GLOBAL);
     try {
       checkOperation(OperationCategory.WRITE);
 
@@ -6037,7 +6041,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       blockManager.setBlockToken(locatedBlock,
           BlockTokenIdentifier.AccessMode.WRITE);
     } finally {
-      writeUnlock("bumpBlockGenerationStamp");
+      writeUnlock(FSNamesystemLockMode.GLOBAL, "bumpBlockGenerationStamp");
     }
     // Ensure we record the new generation stamp
     getEditLog().logSync();
@@ -6066,7 +6070,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
              + ", newNodes=" + Arrays.asList(newNodes)
              + ", client=" + clientName
              + ")");
-    writeLock();
+    writeLock(FSNamesystemLockMode.GLOBAL);
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Pipeline not updated");
@@ -6075,7 +6079,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       updatePipelineInternal(clientName, oldBlock, newBlock, newNodes,
           newStorageIDs, logRetryCache);
     } finally {
-      writeUnlock("updatePipeline");
+      writeUnlock(FSNamesystemLockMode.GLOBAL, "updatePipeline");
     }
     getEditLog().logSync();
     LOG.info("updatePipeline(" + oldBlock.getLocalBlock() + " => "
@@ -6086,7 +6090,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
       ExtendedBlock newBlock, DatanodeID[] newNodes, String[] newStorageIDs,
       boolean logRetryCache)
       throws IOException {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.GLOBAL);
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
     final String src = pendingFile.getFullPathName();
@@ -6382,7 +6386,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     long expiryTime;
     checkOperation(OperationCategory.WRITE);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
 
@@ -6399,7 +6403,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
         getEditLog().logRenewDelegationToken(id, expiryTime);
         tokenId = id.toStringStable();
       } finally {
-        writeUnlock(operationName, getLockReportInfoSupplier(tokenId));
+        writeUnlock(FSNamesystemLockMode.FS, operationName, 
getLockReportInfoSupplier(tokenId));
       }
     } catch (AccessControlException ace) {
       final DelegationTokenIdentifier id = 
DFSUtil.decodeDelegationToken(token);
@@ -6423,7 +6427,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
     String tokenId = null;
     checkOperation(OperationCategory.WRITE);
     try {
-      writeLock();
+      writeLock(FSNamesystemLockMode.FS);
       try {
         checkOperation(OperationCategory.WRITE);
         checkNameNodeSafeMode("Cannot cancel delegation token");
@@ -6433,7 +6437,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
         getEditLog().logCancelDelegationToken(id);
         tokenId = id.toStringStable();
       } finally {
-        writeUnlock(operationName, getLockReportInfoSupplier(tokenId));
+        writeUnlock(FSNamesystemLockMode.FS, operationName, 
getLockReportInfoSupplier(tokenId));
       }
     } catch (AccessControlException ace) {
       final DelegationTokenIdentifier id = 
DFSUtil.decodeDelegationToken(token);
@@ -6508,7 +6512,7 @@ public class FSNamesystem implements Namesystem, 
FSNamesystemMBean,
   
   private void logReassignLease(String leaseHolder, String src,
       String newHolder) {
-    assert hasWriteLock();
+    assert hasWriteLock(FSNamesystemLockMode.FS);
     getEditLog().logReassignLease(leaseHolder, src, newHolder);
   }
   
diff --git 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 1bd315f1771e..5389796b0b1e 100644
--- 
a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ 
b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -972,6 +972,12 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /**
    * Compute file size of the current file.
+   *
+   * ComputeFileSize only needs the FSLock even through it involves block.
+   * BlockSize only be changed by hsync, addBlock, commitBlockSynchronization,
+   * complete, updatePipeline and forceCompleteBlock, all these operations
+   * already hold the FSWriteLock.
+   * CompleteBlock also hold the FSWriteLock since it needs to update Quota
    * 
    * @param includesLastUcBlock
    *          If the last block is under construction, should it be included?


---------------------------------------------------------------------
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