You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by we...@apache.org on 2019/08/29 17:11:55 UTC

[hadoop] branch trunk updated: HDFS-11246. FSNameSystem#logAuditEvent should be called outside the read or write locks. Contributed by He Xiaoqiao, Kuhu Shukla.

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

weichiu pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new f600fbb  HDFS-11246. FSNameSystem#logAuditEvent should be called outside the read or write locks. Contributed by He Xiaoqiao, Kuhu Shukla.
f600fbb is described below

commit f600fbb6c4987c69292faea6b5abf022bb213ffd
Author: He Xiaoqiao <he...@apache.org>
AuthorDate: Thu Aug 29 10:07:44 2019 -0700

    HDFS-11246. FSNameSystem#logAuditEvent should be called outside the read or write locks. Contributed by He Xiaoqiao, Kuhu Shukla.
    
    Signed-off-by: Wei-Chiu Chuang <we...@apache.org>
    Co-authored-by: Kuhu Shukla <ks...@apache.org>
---
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  | 1057 +++++++++++---------
 1 file changed, 558 insertions(+), 499 deletions(-)

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 55722d7..9d6e99e 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
@@ -1823,26 +1823,28 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "listOpenFiles";
     checkSuperuserPrivilege();
     checkOperation(OperationCategory.READ);
-    readLock();
     BatchedListEntries<OpenFileEntry> batchedListEntries;
     try {
-      checkOperation(OperationCategory.READ);
-      if(openFilesTypes.contains(OpenFilesType.ALL_OPEN_FILES)) {
-        batchedListEntries = leaseManager.getUnderConstructionFiles(prevId,
-            path);
-      } else {
-        if(openFilesTypes.contains(OpenFilesType.BLOCKING_DECOMMISSION)) {
-          batchedListEntries = getFilesBlockingDecom(prevId, path);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        if (openFilesTypes.contains(OpenFilesType.ALL_OPEN_FILES)) {
+          batchedListEntries = leaseManager.getUnderConstructionFiles(prevId,
+              path);
         } else {
-          throw new IllegalArgumentException("Unknown OpenFileType: "
-              + openFilesTypes);
+          if (openFilesTypes.contains(OpenFilesType.BLOCKING_DECOMMISSION)) {
+            batchedListEntries = getFilesBlockingDecom(prevId, path);
+          } else {
+            throw new IllegalArgumentException("Unknown OpenFileType: "
+                + openFilesTypes);
+          }
         }
+      } finally {
+        readUnlock(operationName);
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, null);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
     logAuditEvent(true, operationName, null);
     return batchedListEntries;
@@ -1915,16 +1917,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set permission for " + src);
-      auditStat = FSDirAttrOp.setPermission(dir, pc, src, permission);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set permission for " + src);
+        auditStat = FSDirAttrOp.setPermission(dir, pc, src, permission);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -1943,16 +1947,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set owner for " + src);
-      auditStat = FSDirAttrOp.setOwner(dir, pc, src, username, group);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set owner for " + src);
+        auditStat = FSDirAttrOp.setOwner(dir, pc, src, username, group);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -1968,40 +1974,42 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     GetBlockLocationsResult res = null;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      res = FSDirStatAndListingOp.getBlockLocations(
-          dir, pc, srcArg, offset, length, true);
-      if (isInSafeMode()) {
-        for (LocatedBlock b : res.blocks.getLocatedBlocks()) {
-          // if safemode & no block locations yet then throw safemodeException
-          if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
-            SafeModeException se = newSafemodeException(
-                "Zero blocklocations for " + srcArg);
-            if (haEnabled && haContext != null &&
-                (haContext.getState().getServiceState() == ACTIVE ||
-                    haContext.getState().getServiceState() == OBSERVER)) {
-              throw new RetriableException(se);
-            } else {
-              throw se;
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        res = FSDirStatAndListingOp.getBlockLocations(
+            dir, pc, srcArg, offset, length, true);
+        if (isInSafeMode()) {
+          for (LocatedBlock b : res.blocks.getLocatedBlocks()) {
+            // if safemode & no block locations yet then throw safemodeException
+            if ((b.getLocations() == null) || (b.getLocations().length == 0)) {
+              SafeModeException se = newSafemodeException(
+                  "Zero blocklocations for " + srcArg);
+              if (haEnabled && haContext != null &&
+                  (haContext.getState().getServiceState() == ACTIVE ||
+                      haContext.getState().getServiceState() == OBSERVER)) {
+                throw new RetriableException(se);
+              } else {
+                throw se;
+              }
             }
           }
-        }
-      } else if (haEnabled && haContext != null &&
-          haContext.getState().getServiceState() == OBSERVER) {
-        for (LocatedBlock b : res.blocks.getLocatedBlocks()) {
-          if (b.getLocations() == null || b.getLocations().length == 0) {
-            throw new ObserverRetryOnActiveException("Zero blocklocations for "
-                + srcArg);
+        } else if (haEnabled && haContext != null &&
+            haContext.getState().getServiceState() == OBSERVER) {
+          for (LocatedBlock b : res.blocks.getLocatedBlocks()) {
+            if (b.getLocations() == null || b.getLocations().length == 0) {
+              throw new ObserverRetryOnActiveException("Zero blocklocations "
+                  + "for " + srcArg);
+            }
           }
         }
+      } finally {
+        readUnlock(operationName);
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, srcArg);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
 
     logAuditEvent(true, operationName, srcArg);
@@ -2009,44 +2017,46 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (!isInSafeMode() && res.updateAccessTime()) {
       String src = srcArg;
       checkOperation(OperationCategory.WRITE);
-      writeLock();
-      final long now = now();
       try {
-        checkOperation(OperationCategory.WRITE);
-        /**
-         * Resolve the path again and update the atime only when the file
-         * exists.
-         *
-         * XXX: Races can still occur even after resolving the path again.
-         * For example:
-         *
-         * <ul>
-         *   <li>Get the block location for "/a/b"</li>
-         *   <li>Rename "/a/b" to "/c/b"</li>
-         *   <li>The second resolution still points to "/a/b", which is
-         *   wrong.</li>
-         * </ul>
-         *
-         * The behavior is incorrect but consistent with the one before
-         * HDFS-7463. A better fix is to change the edit log of SetTime to
-         * use inode id instead of a path.
-         */
-        final INodesInPath iip = dir.resolvePath(pc, srcArg, DirOp.READ);
-        src = iip.getPath();
-
-        INode inode = iip.getLastINode();
-        boolean updateAccessTime = inode != null &&
-            now > inode.getAccessTime() + dir.getAccessTimePrecision();
-        if (!isInSafeMode() && updateAccessTime) {
-          boolean changed = FSDirAttrOp.setTimes(dir, iip, -1, now, false);
-          if (changed) {
-            getEditLog().logTimes(src, -1, now);
+        writeLock();
+        final long now = now();
+        try {
+          checkOperation(OperationCategory.WRITE);
+          /**
+           * Resolve the path again and update the atime only when the file
+           * exists.
+           *
+           * XXX: Races can still occur even after resolving the path again.
+           * For example:
+           *
+           * <ul>
+           *   <li>Get the block location for "/a/b"</li>
+           *   <li>Rename "/a/b" to "/c/b"</li>
+           *   <li>The second resolution still points to "/a/b", which is
+           *   wrong.</li>
+           * </ul>
+           *
+           * The behavior is incorrect but consistent with the one before
+           * HDFS-7463. A better fix is to change the edit log of SetTime to
+           * use inode id instead of a path.
+           */
+          final INodesInPath iip = dir.resolvePath(pc, srcArg, DirOp.READ);
+          src = iip.getPath();
+
+          INode inode = iip.getLastINode();
+          boolean updateAccessTime = inode != null &&
+              now > inode.getAccessTime() + dir.getAccessTimePrecision();
+          if (!isInSafeMode() && updateAccessTime) {
+            boolean changed = FSDirAttrOp.setTimes(dir, iip, -1, now, false);
+            if (changed) {
+              getEditLog().logTimes(src, -1, now);
+            }
           }
+        } finally {
+          writeUnlock(operationName);
         }
       } catch (Throwable e) {
         LOG.warn("Failed to update the access time of " + src, e);
-      } finally {
-        writeUnlock(operationName);
       }
     }
 
@@ -2089,26 +2099,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "concat";
     FileStatus stat = null;
-    boolean success = false;
     final FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot concat " + target);
-      stat = FSDirConcatOp.concat(dir, pc, target, srcs, logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot concat " + target);
+        stat = FSDirConcatOp.concat(dir, pc, target, srcs, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, Arrays.toString(srcs),
+      logAuditEvent(false, operationName, Arrays.toString(srcs),
           target, stat);
       throw ace;
     } finally {
-      writeUnlock(operationName);
-      if (success) {
-        getEditLog().logSync();
-      }
+      getEditLog().logSync();
     }
-    logAuditEvent(success, operationName, Arrays.toString(srcs), target, stat);
+    logAuditEvent(true, operationName, Arrays.toString(srcs), target, stat);
   }
 
   /**
@@ -2121,16 +2130,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set times " + src);
-      auditStat = FSDirAttrOp.setTimes(dir, pc, src, mtime, atime);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set times " + src);
+        auditStat = FSDirAttrOp.setTimes(dir, pc, src, mtime, atime);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -2196,17 +2207,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot create symlink " + link);
-      auditStat = FSDirSymlinkOp.createSymlinkInt(this, target, link, dirPerms,
-                                                  createParent, logRetryCache);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot create symlink " + link);
+        auditStat = FSDirSymlinkOp.createSymlinkInt(this, target, link,
+            dirPerms, createParent, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, link, target, null);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, link, target, auditStat);
@@ -2232,17 +2245,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean success = false;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set replication for " + src);
-      success = FSDirAttrOp.setReplication(dir, pc, blockManager, src,
-          replication);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set replication for " + src);
+        success = FSDirAttrOp.setReplication(dir, pc, blockManager, src,
+            replication);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     if (success) {
       getEditLog().logSync();
@@ -2263,17 +2278,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set storage policy for " + src);
-      auditStat = FSDirAttrOp.setStoragePolicy(dir, pc, blockManager, src,
-                                               policyName);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set storage policy for " + src);
+        auditStat = FSDirAttrOp.setStoragePolicy(dir, pc, blockManager, src,
+            policyName);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -2291,17 +2308,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat;
     validateStoragePolicySatisfy();
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot satisfy storage policy for " + src);
-      auditStat = FSDirSatisfyStoragePolicyOp.satisfyStoragePolicy(
-          dir, blockManager, src, logRetryCache);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot satisfy storage policy for " + src);
+        auditStat = FSDirSatisfyStoragePolicyOp.satisfyStoragePolicy(
+            dir, blockManager, src, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -2341,16 +2360,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot unset storage policy for " + src);
-      auditStat = FSDirAttrOp.unsetStoragePolicy(dir, pc, blockManager, src);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot unset storage policy for " + src);
+        auditStat = FSDirAttrOp.unsetStoragePolicy(dir, pc, blockManager, src);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -3039,16 +3060,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FSDirRenameOp.RenameResult ret = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot rename " + src);
-      ret = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot rename " + src);
+        ret = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e)  {
       logAuditEvent(false, operationName, src, dst, null);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     boolean success = ret.success;
     if (success) {
@@ -3065,20 +3088,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FSDirRenameOp.RenameResult res = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot rename " + src);
-      res = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache,
-          options);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot rename " + src);
+        res = FSDirRenameOp.renameToInt(dir, pc, src, dst, logRetryCache,
+            options);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName + " (options=" +
           Arrays.toString(options) + ")", src, dst, null);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
-
     getEditLog().logSync();
 
     BlocksMapUpdateInfo collectedBlocks = res.collectedBlocks;
@@ -3103,19 +3127,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     BlocksMapUpdateInfo toRemovedBlocks = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     boolean ret = false;
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot delete " + src);
-      toRemovedBlocks = FSDirDeleteOp.delete(
-          this, pc, src, recursive, logRetryCache);
-      ret = toRemovedBlocks != null;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot delete " + src);
+        toRemovedBlocks = FSDirDeleteOp.delete(
+            this, pc, src, recursive, logRetryCache);
+        ret = toRemovedBlocks != null;
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     if (toRemovedBlocks != null) {
@@ -3209,16 +3235,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     HdfsFileStatus stat = null;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      stat = FSDirStatAndListingOp.getFileInfo(
-          dir, pc, src, resolveLink, needLocation, needBlockToken);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        stat = FSDirStatAndListingOp.getFileInfo(
+            dir, pc, src, resolveLink, needLocation, needBlockToken);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
     logAuditEvent(true, operationName, src);
     return stat;
@@ -3231,15 +3259,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "isFileClosed";
     checkOperation(OperationCategory.READ);
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      return FSDirStatAndListingOp.isFileClosed(dir, pc, src);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        return FSDirStatAndListingOp.isFileClosed(dir, pc, src);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
   }
 
@@ -3252,17 +3282,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot create directory " + src);
-      auditStat = FSDirMkdirOp.mkdirs(this, pc, src, permissions,
-          createParent);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot create directory " + src);
+        auditStat = FSDirMkdirOp.mkdirs(this, pc, src, permissions,
+            createParent);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -3286,21 +3318,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   ContentSummary getContentSummary(final String src) throws IOException {
     checkOperation(OperationCategory.READ);
     final String operationName = "contentSummary";
-    boolean success = true;
     ContentSummary cs;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      cs = FSDirStatAndListingOp.getContentSummary(dir, pc, src);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        cs = FSDirStatAndListingOp.getContentSummary(dir, pc, src);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      success = false;
-      logAuditEvent(success, operationName, src);
+      logAuditEvent(false, operationName, src);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-    logAuditEvent(success, operationName, src);
+    logAuditEvent(true, operationName, src);
     return cs;
   }
 
@@ -3323,19 +3355,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "quotaUsage";
     QuotaUsage quotaUsage;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
-    boolean success = true;
     try {
-      checkOperation(OperationCategory.READ);
-      quotaUsage = FSDirStatAndListingOp.getQuotaUsage(dir, pc, src);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        quotaUsage = FSDirStatAndListingOp.getQuotaUsage(dir, pc, src);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      success = false;
-      logAuditEvent(success, operationName, src);
+      logAuditEvent(false, operationName, src);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-    logAuditEvent(success, operationName, src);
+    logAuditEvent(true, operationName, src);
     return quotaUsage;
   }
 
@@ -3354,23 +3386,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final String operationName = getQuotaCommand(nsQuota, ssQuota);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
-    boolean success = false;
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set quota on " + src);
-      FSDirAttrOp.setQuota(dir, pc, src, nsQuota, ssQuota, type);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set quota on " + src);
+        FSDirAttrOp.setQuota(dir, pc, src, nsQuota, ssQuota, type);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, src);
+      logAuditEvent(false, operationName, src);
       throw ace;
     } finally {
-      writeUnlock(operationName);
-      if (success) {
-        getEditLog().logSync();
-      }
+      getEditLog().logSync();
     }
-    logAuditEvent(success, operationName, src);
+    logAuditEvent(true, operationName, src);
   }
 
   /** Persist all metadata about this file.
@@ -3888,15 +3919,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "listStatus";
     DirectoryListing dl = null;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(NameNode.OperationCategory.READ);
-      dl = getListingInt(dir, pc, src, startAfter, needLocation);
+      readLock();
+      try {
+        checkOperation(NameNode.OperationCategory.READ);
+        dl = getListingInt(dir, pc, src, startAfter, needLocation);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
     logAuditEvent(true, operationName, src);
     return dl;
@@ -4494,7 +4527,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       final DatanodeManager dm = getBlockManager().getDatanodeManager();      
       reports = dm.getDatanodeStorageReport(type);
     } finally {
-      readUnlock("getDatanodeStorageReport");
+      readUnlock(operationName);
     }
     logAuditEvent(true, operationName, null);
     return reports;
@@ -5742,7 +5775,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       tokenId = dtId.toStringStable();
       success = true;
     } finally {
-      writeUnlock("getDelegationToken");
+      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(success, operationName, tokenId);
@@ -5759,35 +5792,37 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   long renewDelegationToken(Token<DelegationTokenIdentifier> token)
       throws InvalidToken, IOException {
     final String operationName = "renewDelegationToken";
-    boolean success = false;
     String tokenId;
     long expiryTime;
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
 
-      checkNameNodeSafeMode("Cannot renew delegation token");
-      if (!isAllowedDelegationTokenOp()) {
-        throw new IOException(
-            "Delegation Token can be renewed only with kerberos or web authentication");
+        checkNameNodeSafeMode("Cannot renew delegation token");
+        if (!isAllowedDelegationTokenOp()) {
+          throw new IOException(
+              "Delegation Token can be renewed only with kerberos or web "
+                  + "authentication");
+        }
+        String renewer = getRemoteUser().getShortUserName();
+        expiryTime = dtSecretManager.renewToken(token, renewer);
+        final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(
+            token);
+        getEditLog().logRenewDelegationToken(id, expiryTime);
+        tokenId = id.toStringStable();
+      } finally {
+        writeUnlock(operationName);
       }
-      String renewer = getRemoteUser().getShortUserName();
-      expiryTime = dtSecretManager.renewToken(token, renewer);
-      final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
-      getEditLog().logRenewDelegationToken(id, expiryTime);
-      tokenId = id.toStringStable();
-      success = true;
     } catch (AccessControlException ace) {
       final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
       tokenId = id.toStringStable();
-      logAuditEvent(success, operationName, tokenId);
+      logAuditEvent(false, operationName, tokenId);
       throw ace;
-    } finally {
-      writeUnlock("renewDelegationToken");
     }
     getEditLog().logSync();
-    logAuditEvent(success, operationName, tokenId);
+    logAuditEvent(true, operationName, tokenId);
     return expiryTime;
   }
 
@@ -5799,30 +5834,29 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void cancelDelegationToken(Token<DelegationTokenIdentifier> token)
       throws IOException {
     final String operationName = "cancelDelegationToken";
-    boolean success = false;
     String tokenId;
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-
-      checkNameNodeSafeMode("Cannot cancel delegation token");
-      String canceller = getRemoteUser().getUserName();
-      DelegationTokenIdentifier id = dtSecretManager
-        .cancelToken(token, canceller);
-      getEditLog().logCancelDelegationToken(id);
-      tokenId = id.toStringStable();
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot cancel delegation token");
+        String canceller = getRemoteUser().getUserName();
+        DelegationTokenIdentifier id = dtSecretManager
+            .cancelToken(token, canceller);
+        getEditLog().logCancelDelegationToken(id);
+        tokenId = id.toStringStable();
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
       final DelegationTokenIdentifier id = DFSUtil.decodeDelegationToken(token);
       tokenId = id.toStringStable();
-      logAuditEvent(success, operationName, tokenId);
+      logAuditEvent(false, operationName, tokenId);
       throw ace;
-    } finally {
-      writeUnlock("cancelDelegationToken");
     }
     getEditLog().logSync();
-    logAuditEvent(success, operationName, tokenId);
+    logAuditEvent(true, operationName, tokenId);
   }
 
   /**
@@ -6538,18 +6572,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final String operationName = "disallowSnapshot";
     checkSuperuserPrivilege(operationName);
-    boolean success = false;
     writeLock();
     try {
       checkOperation(OperationCategory.WRITE);
       checkNameNodeSafeMode("Cannot disallow snapshot for " + path);
       FSDirSnapshotOp.disallowSnapshot(dir, snapshotManager, path);
-      success = true;
     } finally {
       writeUnlock(operationName);
     }
     getEditLog().logSync();
-    logAuditEvent(success, operationName, path, null, null);
+    logAuditEvent(true, operationName, path, null, null);
   }
   
   /**
@@ -6562,24 +6594,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     final String operationName = "createSnapshot";
     String snapshotPath = null;
-    boolean success = false;
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot create snapshot for " + snapshotRoot);
-      snapshotPath = FSDirSnapshotOp.createSnapshot(dir, pc,
-          snapshotManager, snapshotRoot, snapshotName, logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot create snapshot for " + snapshotRoot);
+        snapshotPath = FSDirSnapshotOp.createSnapshot(dir, pc,
+            snapshotManager, snapshotRoot, snapshotName, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, snapshotRoot,
+      logAuditEvent(false, operationName, snapshotRoot,
           snapshotPath, null);
       throw ace;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
-    logAuditEvent(success, operationName, snapshotRoot,
+    logAuditEvent(true, operationName, snapshotRoot,
         snapshotPath, null);
     return snapshotPath;
   }
@@ -6597,26 +6629,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       boolean logRetryCache) throws IOException {
     checkOperation(OperationCategory.WRITE);
     final String operationName = "renameSnapshot";
-    boolean success = false;
     String oldSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotOldName);
     String newSnapshotRoot = Snapshot.getSnapshotPath(path, snapshotNewName);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot rename snapshot for " + path);
-      FSDirSnapshotOp.renameSnapshot(dir, pc, snapshotManager, path,
-          snapshotOldName, snapshotNewName, logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot rename snapshot for " + path);
+        FSDirSnapshotOp.renameSnapshot(dir, pc, snapshotManager, path,
+            snapshotOldName, snapshotNewName, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, oldSnapshotRoot,
+      logAuditEvent(false, operationName, oldSnapshotRoot,
           newSnapshotRoot, null);
       throw ace;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
-    logAuditEvent(success, operationName, oldSnapshotRoot,
+    logAuditEvent(true, operationName, oldSnapshotRoot,
         newSnapshotRoot, null);
   }
 
@@ -6632,21 +6664,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "listSnapshottableDirectory";
     SnapshottableDirectoryStatus[] status = null;
     checkOperation(OperationCategory.READ);
-    boolean success = false;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      status = FSDirSnapshotOp.getSnapshottableDirListing(dir, pc,
-          snapshotManager);
-      success = true;
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        status = FSDirSnapshotOp.getSnapshottableDirListing(dir, pc,
+            snapshotManager);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, null, null, null);
+      logAuditEvent(false, operationName, null, null, null);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-    logAuditEvent(success, operationName, null, null, null);
+    logAuditEvent(true, operationName, null, null, null);
     return status;
   }
   
@@ -6671,27 +6703,26 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "computeSnapshotDiff";
     SnapshotDiffReport diffs = null;
     checkOperation(OperationCategory.READ);
-    boolean success = false;
     String fromSnapshotRoot = (fromSnapshot == null || fromSnapshot.isEmpty()) ?
         path : Snapshot.getSnapshotPath(path, fromSnapshot);
     String toSnapshotRoot = (toSnapshot == null || toSnapshot.isEmpty()) ?
         path : Snapshot.getSnapshotPath(path, toSnapshot);
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     long actualTime = Time.monotonicNow();
     try {
-      checkOperation(OperationCategory.READ);
-      diffs = FSDirSnapshotOp.getSnapshotDiffReport(dir, pc, snapshotManager,
-          path, fromSnapshot, toSnapshot);
-      success = true;
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        diffs = FSDirSnapshotOp.getSnapshotDiffReport(dir, pc, snapshotManager,
+            path, fromSnapshot, toSnapshot);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, fromSnapshotRoot,
+      logAuditEvent(false, operationName, fromSnapshotRoot,
           toSnapshotRoot, null);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-
     if (diffs != null) {
       SnapshotDiffReport.DiffStats dstat = diffs.getStats();
       LOG.info("SnapshotDiffReport '"
@@ -6708,7 +6739,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           + ((Time.monotonicNow() - begTime) / 1000.0) + "s.");
     }
 
-    logAuditEvent(success, operationName, fromSnapshotRoot,
+    logAuditEvent(true, operationName, fromSnapshotRoot,
         toSnapshotRoot, null);
     return diffs;
   }
@@ -6743,7 +6774,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "computeSnapshotDiff";
     SnapshotDiffReportListing diffs = null;
     checkOperation(OperationCategory.READ);
-    boolean success = false;
     String fromSnapshotRoot =
         (fromSnapshot == null || fromSnapshot.isEmpty()) ? path :
             Snapshot.getSnapshotPath(path, fromSnapshot);
@@ -6751,22 +6781,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         (toSnapshot == null || toSnapshot.isEmpty()) ? path :
             Snapshot.getSnapshotPath(path, toSnapshot);
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      diffs = FSDirSnapshotOp
-          .getSnapshotDiffReportListing(dir, pc, snapshotManager, path,
-              fromSnapshot, toSnapshot, startPath, index,
-              snapshotDiffReportLimit);
-      success = true;
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        diffs = FSDirSnapshotOp
+            .getSnapshotDiffReportListing(dir, pc, snapshotManager, path,
+                fromSnapshot, toSnapshot, startPath, index,
+                snapshotDiffReportLimit);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, fromSnapshotRoot, toSnapshotRoot,
+      logAuditEvent(false, operationName, fromSnapshotRoot, toSnapshotRoot,
           null);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-    logAuditEvent(success, operationName, fromSnapshotRoot, toSnapshotRoot,
+    logAuditEvent(true, operationName, fromSnapshotRoot, toSnapshotRoot,
         null);
     return diffs;
   }
@@ -6781,24 +6812,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void deleteSnapshot(String snapshotRoot, String snapshotName,
       boolean logRetryCache) throws IOException {
     final String operationName = "deleteSnapshot";
-    boolean success = false;
     String rootPath = null;
     BlocksMapUpdateInfo blocksToBeDeleted = null;
     final FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot);
-      rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
-      blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(dir, pc,
-          snapshotManager, snapshotRoot, snapshotName, logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot delete snapshot for " + snapshotRoot);
+        rootPath = Snapshot.getSnapshotPath(snapshotRoot, snapshotName);
+        blocksToBeDeleted = FSDirSnapshotOp.deleteSnapshot(dir, pc,
+            snapshotManager, snapshotRoot, snapshotName, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, rootPath, null, null);
+      logAuditEvent(false, operationName, rootPath, null, null);
       throw ace;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
 
@@ -6807,7 +6838,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     if (blocksToBeDeleted != null) {
       removeBlocks(blocksToBeDeleted);
     }
-    logAuditEvent(success, operationName, rootPath, null, null);
+    logAuditEvent(true, operationName, rootPath, null, null);
   }
 
   /**
@@ -7074,31 +7105,30 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "addCacheDirective";
     CacheDirectiveInfo effectiveDirective = null;
-    boolean success = false;
     String effectiveDirectiveStr;
     if (!flags.contains(CacheFlag.FORCE)) {
       cacheManager.waitForRescanIfNeeded();
     }
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot add cache directive");
-      effectiveDirective = FSNDNCacheOp.addCacheDirective(this, cacheManager,
-          directive, flags, logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot add cache directive");
+        effectiveDirective = FSNDNCacheOp.addCacheDirective(this, cacheManager,
+            directive, flags, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, null,
+      logAuditEvent(false, operationName, null,
           null, null);
       throw ace;
     } finally {
-      writeUnlock(operationName);
-      if (success) {
-        getEditLog().logSync();
-      }
+      getEditLog().logSync();
     }
     effectiveDirectiveStr = effectiveDirective.toString();
-    logAuditEvent(success, operationName, effectiveDirectiveStr,
+    logAuditEvent(true, operationName, effectiveDirectiveStr,
         null, null);
     return effectiveDirective.getId();
   }
@@ -7106,51 +7136,51 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   void modifyCacheDirective(CacheDirectiveInfo directive,
       EnumSet<CacheFlag> flags, boolean logRetryCache) throws IOException {
     final String operationName = "modifyCacheDirective";
-    boolean success = false;
     final String idStr = "{id: " + directive.getId() + "}";
     if (!flags.contains(CacheFlag.FORCE)) {
       cacheManager.waitForRescanIfNeeded();
     }
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot add cache directive");
-      FSNDNCacheOp.modifyCacheDirective(this, cacheManager, directive, flags,
-          logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot add cache directive");
+        FSNDNCacheOp.modifyCacheDirective(this, cacheManager, directive, flags,
+            logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, idStr,
+      logAuditEvent(false, operationName, idStr,
           directive.toString(), null);
       throw ace;
     } finally {
-      writeUnlock(operationName);
-      if (success) {
-        getEditLog().logSync();
-      }
+      getEditLog().logSync();
     }
-    logAuditEvent(success, operationName, idStr,
+    logAuditEvent(true, operationName, idStr,
         directive.toString(), null);
   }
 
   void removeCacheDirective(long id, boolean logRetryCache) throws IOException {
     final String operationName = "removeCacheDirective";
-    boolean success = false;
     String idStr = "{id: " + Long.toString(id) + "}";
     checkOperation(OperationCategory.WRITE);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot remove cache directives");
-      FSNDNCacheOp.removeCacheDirective(this, cacheManager, id, logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot remove cache directives");
+        FSNDNCacheOp.removeCacheDirective(this, cacheManager, id,
+            logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, idStr, null, null);
+      logAuditEvent(false, operationName, idStr, null, null);
       throw ace;
-    } finally {
-      writeUnlock(operationName);
     }
-    logAuditEvent(success, operationName, idStr, null, null);
+    logAuditEvent(true, operationName, idStr, null, null);
     getEditLog().logSync();
   }
 
@@ -7160,21 +7190,21 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     BatchedListEntries<CacheDirectiveEntry> results;
     cacheManager.waitForRescanIfNeeded();
-    readLock();
-    boolean success = false;
     try {
-      checkOperation(OperationCategory.READ);
-      results = FSNDNCacheOp.listCacheDirectives(this, cacheManager, startId,
-          filter);
-      success = true;
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        results = FSNDNCacheOp.listCacheDirectives(this, cacheManager, startId,
+            filter);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, filter.toString(), null,
+      logAuditEvent(false, operationName, filter.toString(), null,
           null);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-    logAuditEvent(success, operationName, filter.toString(), null,
+    logAuditEvent(true, operationName, filter.toString(), null,
         null);
     return results;
   }
@@ -7183,24 +7213,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "addCachePool";
     checkOperation(OperationCategory.WRITE);
-    writeLock();
-    boolean success = false;
     String poolInfoStr = null;
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot add cache pool"
-          + (req == null ? null : req.getPoolName()));
-      CachePoolInfo info = FSNDNCacheOp.addCachePool(this, cacheManager, req,
-          logRetryCache);
-      poolInfoStr = info.toString();
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot add cache pool"
+            + (req == null ? null : req.getPoolName()));
+        CachePoolInfo info = FSNDNCacheOp.addCachePool(this, cacheManager, req,
+            logRetryCache);
+        poolInfoStr = info.toString();
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, poolInfoStr, null, null);
+      logAuditEvent(false, operationName, poolInfoStr, null, null);
       throw ace;
-    } finally {
-      writeUnlock(operationName);
     }
-    logAuditEvent(success, operationName, poolInfoStr, null, null);
+    logAuditEvent(true, operationName, poolInfoStr, null, null);
     getEditLog().logSync();
   }
 
@@ -7208,24 +7238,24 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "modifyCachePool";
     checkOperation(OperationCategory.WRITE);
-    writeLock();
-    boolean success = false;
     String poolNameStr = "{poolName: " +
         (req == null ? null : req.getPoolName()) + "}";
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot modify cache pool"
-          + (req == null ? null : req.getPoolName()));
-      FSNDNCacheOp.modifyCachePool(this, cacheManager, req, logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot modify cache pool"
+            + (req == null ? null : req.getPoolName()));
+        FSNDNCacheOp.modifyCachePool(this, cacheManager, req, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, poolNameStr,
+      logAuditEvent(false, operationName, poolNameStr,
           req == null ? null : req.toString(), null);
       throw ace;
-    } finally {
-      writeUnlock(operationName);
     }
-    logAuditEvent(success, operationName, poolNameStr,
+    logAuditEvent(true, operationName, poolNameStr,
         req == null ? null : req.toString(), null);
 
     getEditLog().logSync();
@@ -7235,22 +7265,22 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throws IOException {
     final String operationName = "removeCachePool";
     checkOperation(OperationCategory.WRITE);
-    writeLock();
-    boolean success = false;
     String poolNameStr = "{poolName: " + cachePoolName + "}";
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot modify cache pool" + cachePoolName);
-      FSNDNCacheOp.removeCachePool(this, cacheManager, cachePoolName,
-          logRetryCache);
-      success = true;
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot modify cache pool" + cachePoolName);
+        FSNDNCacheOp.removeCachePool(this, cacheManager, cachePoolName,
+            logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, poolNameStr, null, null);
+      logAuditEvent(false, operationName, poolNameStr, null, null);
       throw ace;
-    } finally {
-      writeUnlock(operationName);
     }
-    logAuditEvent(success, operationName, poolNameStr, null, null);
+    logAuditEvent(true, operationName, poolNameStr, null, null);
     getEditLog().logSync();
   }
 
@@ -7259,20 +7289,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "listCachePools";
     BatchedListEntries<CachePoolEntry> results;
     checkOperation(OperationCategory.READ);
-    boolean success = false;
     cacheManager.waitForRescanIfNeeded();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      results = FSNDNCacheOp.listCachePools(this, cacheManager, prevKey);
-      success = true;
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        results = FSNDNCacheOp.listCachePools(this, cacheManager, prevKey);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, null, null, null);
+      logAuditEvent(false, operationName, null, null, null);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-    logAuditEvent(success, operationName, null, null, null);
+    logAuditEvent(true, operationName, null, null, null);
     return results;
   }
 
@@ -7282,16 +7312,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
-      auditStat = FSDirAclOp.modifyAclEntries(dir, pc, src, aclSpec);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot modify ACL entries on " + src);
+        auditStat = FSDirAclOp.modifyAclEntries(dir, pc, src, aclSpec);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -7303,16 +7335,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     FileStatus auditStat = null;
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
-      auditStat = FSDirAclOp.removeAclEntries(dir, pc, src, aclSpec);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot remove ACL entries on " + src);
+        auditStat = FSDirAclOp.removeAclEntries(dir, pc, src, aclSpec);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -7323,16 +7357,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
-      auditStat = FSDirAclOp.removeDefaultAcl(dir, pc, src);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot remove default ACL entries on " + src);
+        auditStat = FSDirAclOp.removeDefaultAcl(dir, pc, src);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -7343,16 +7379,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot remove ACL on " + src);
-      auditStat = FSDirAclOp.removeAcl(dir, pc, src);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot remove ACL on " + src);
+        auditStat = FSDirAclOp.removeAcl(dir, pc, src);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -7363,16 +7401,18 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set ACL on " + src);
-      auditStat = FSDirAclOp.setAcl(dir, pc, src, aclSpec);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set ACL on " + src);
+        auditStat = FSDirAclOp.setAcl(dir, pc, src, aclSpec);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -7383,15 +7423,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     final AclStatus ret;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      ret = FSDirAclOp.getAclStatus(dir, pc, src);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        ret = FSDirAclOp.getAclStatus(dir, pc, src);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch(AccessControlException ace) {
       logAuditEvent(false, operationName, src);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
     logAuditEvent(true, operationName, src);
     return ret;
@@ -7449,25 +7491,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     throws AccessControlException, UnresolvedLinkException, IOException {
     final String operationName = "getEZForPath";
     FileStatus resultingStat = null;
-    boolean success = false;
     EncryptionZone encryptionZone;
     final FSPermissionChecker pc = getPermissionChecker();
     checkOperation(OperationCategory.READ);
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      Entry<EncryptionZone, FileStatus> ezForPath = FSDirEncryptionZoneOp
-          .getEZForPath(dir, srcArg, pc);
-      success = true;
-      resultingStat = ezForPath.getValue();
-      encryptionZone = ezForPath.getKey();
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        Entry<EncryptionZone, FileStatus> ezForPath = FSDirEncryptionZoneOp
+            .getEZForPath(dir, srcArg, pc);
+        resultingStat = ezForPath.getValue();
+        encryptionZone = ezForPath.getKey();
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
-      logAuditEvent(success, operationName, srcArg, null, resultingStat);
+      logAuditEvent(false, operationName, srcArg, null, resultingStat);
       throw ace;
-    } finally {
-      readUnlock(operationName);
     }
-    logAuditEvent(success, operationName, srcArg, null, resultingStat);
+    logAuditEvent(true, operationName, srcArg, null, resultingStat);
     return encryptionZone;
   }
 
@@ -7706,17 +7748,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.WRITE);
     checkErasureCodingSupported(operationName);
     boolean success = false;
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot enable erasure coding policy "
-          + ecPolicyName);
-      success = FSDirErasureCodingOp.enableErasureCodingPolicy(this,
-          ecPolicyName, logRetryCache);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot enable erasure coding policy "
+            + ecPolicyName);
+        success = FSDirErasureCodingOp.enableErasureCodingPolicy(this,
+            ecPolicyName, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
       logAuditEvent(false, operationName, ecPolicyName, null, null);
     } finally {
-      writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
         logAuditEvent(success, operationName, ecPolicyName, null, null);
@@ -7739,17 +7784,20 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkErasureCodingSupported(operationName);
     boolean success = false;
     LOG.info("Disable the erasure coding policy " + ecPolicyName);
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot disable erasure coding policy "
-          + ecPolicyName);
-      success = FSDirErasureCodingOp.disableErasureCodingPolicy(this,
-          ecPolicyName, logRetryCache);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot disable erasure coding policy "
+            + ecPolicyName);
+        success = FSDirErasureCodingOp.disableErasureCodingPolicy(this,
+            ecPolicyName, logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException ace) {
       logAuditEvent(false, operationName, ecPolicyName, null, null);
     } finally {
-      writeUnlock(operationName);
       if (success) {
         getEditLog().logSync();
         logAuditEvent(success, operationName, ecPolicyName, null, null);
@@ -7862,17 +7910,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot set XAttr on " + src);
-      auditStat = FSDirXAttrOp.setXAttr(dir, pc, src, xAttr, flag,
-          logRetryCache);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot set XAttr on " + src);
+        auditStat = FSDirXAttrOp.setXAttr(dir, pc, src, xAttr, flag,
+            logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -7884,15 +7934,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     List<XAttr> fsXattrs;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      fsXattrs = FSDirXAttrOp.getXAttrs(dir, pc, src, xAttrs);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        fsXattrs = FSDirXAttrOp.getXAttrs(dir, pc, src, xAttrs);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
     logAuditEvent(true, operationName, src);
     return fsXattrs;
@@ -7903,15 +7955,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     checkOperation(OperationCategory.READ);
     List<XAttr> fsXattrs;
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      fsXattrs = FSDirXAttrOp.listXAttrs(dir, pc, src);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        fsXattrs = FSDirXAttrOp.listXAttrs(dir, pc, src);
+      } finally {
+        readUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
     logAuditEvent(true, operationName, src);
     return fsXattrs;
@@ -7923,16 +7977,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     FileStatus auditStat = null;
     checkOperation(OperationCategory.WRITE);
     final FSPermissionChecker pc = getPermissionChecker();
-    writeLock();
     try {
-      checkOperation(OperationCategory.WRITE);
-      checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
-      auditStat = FSDirXAttrOp.removeXAttr(dir, pc, src, xAttr, logRetryCache);
+      writeLock();
+      try {
+        checkOperation(OperationCategory.WRITE);
+        checkNameNodeSafeMode("Cannot remove XAttr entry on " + src);
+        auditStat = FSDirXAttrOp.removeXAttr(dir, pc, src, xAttr,
+            logRetryCache);
+      } finally {
+        writeUnlock(operationName);
+      }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      writeUnlock(operationName);
     }
     getEditLog().logSync();
     logAuditEvent(true, operationName, src, null, auditStat);
@@ -7965,23 +8022,25 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final String operationName = "checkAccess";
     checkOperation(OperationCategory.READ);
     final FSPermissionChecker pc = getPermissionChecker();
-    readLock();
     try {
-      checkOperation(OperationCategory.READ);
-      final INodesInPath iip = dir.resolvePath(pc, src, DirOp.READ);
-      src = iip.getPath();
-      INode inode = iip.getLastINode();
-      if (inode == null) {
-        throw new FileNotFoundException("Path not found");
-      }
-      if (isPermissionEnabled) {
-        dir.checkPathAccess(pc, iip, mode);
+      readLock();
+      try {
+        checkOperation(OperationCategory.READ);
+        final INodesInPath iip = dir.resolvePath(pc, src, DirOp.READ);
+        src = iip.getPath();
+        INode inode = iip.getLastINode();
+        if (inode == null) {
+          throw new FileNotFoundException("Path not found");
+        }
+        if (isPermissionEnabled) {
+          dir.checkPathAccess(pc, iip, mode);
+        }
+      } finally {
+        readUnlock(operationName);
       }
     } catch (AccessControlException e) {
       logAuditEvent(false, operationName, src);
       throw e;
-    } finally {
-      readUnlock(operationName);
     }
   }
 


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