Selaa lähdekoodia

HDFS-7543. Avoid path resolution when getting FileStatus for audit logs. Contributed by Haohui Mai.

Haohui Mai 10 vuotta sitten
vanhempi
commit
65f2a4ee60

+ 3 - 0
hadoop-hdfs-project/hadoop-hdfs/CHANGES.txt

@@ -471,6 +471,9 @@ Release 2.7.0 - UNRELEASED
     HDFS-7373. Clean up temporary files after fsimage transfer failures.
     HDFS-7373. Clean up temporary files after fsimage transfer failures.
     (kihwal)
     (kihwal)
 
 
+    HDFS-7543. Avoid path resolution when getting FileStatus for audit logs.
+    (wheat9)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.
     HDFS-7454. Reduce memory footprint for AclEntries in NameNode.

+ 15 - 13
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java

@@ -41,10 +41,10 @@ class FSDirAclOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsd.resolvePath(pc, src, pathComponents);
     src = fsd.resolvePath(pc, src, pathComponents);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
-      INodesInPath iip = fsd.getINodesInPath4Write(
-          FSDirectory.normalizePath(src), true);
+      iip = fsd.getINodesInPath4Write(FSDirectory.normalizePath(src), true);
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
       INode inode = FSDirectory.resolveLastINode(iip);
       INode inode = FSDirectory.resolveLastINode(iip);
       int snapshotId = iip.getLatestSnapshotId();
       int snapshotId = iip.getLatestSnapshotId();
@@ -56,7 +56,7 @@ class FSDirAclOp {
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static HdfsFileStatus removeAclEntries(
   static HdfsFileStatus removeAclEntries(
@@ -67,10 +67,10 @@ class FSDirAclOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsd.resolvePath(pc, src, pathComponents);
     src = fsd.resolvePath(pc, src, pathComponents);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
-      INodesInPath iip = fsd.getINodesInPath4Write(
-          FSDirectory.normalizePath(src), true);
+      iip = fsd.getINodesInPath4Write(FSDirectory.normalizePath(src), true);
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
       INode inode = FSDirectory.resolveLastINode(iip);
       INode inode = FSDirectory.resolveLastINode(iip);
       int snapshotId = iip.getLatestSnapshotId();
       int snapshotId = iip.getLatestSnapshotId();
@@ -82,7 +82,7 @@ class FSDirAclOp {
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static HdfsFileStatus removeDefaultAcl(FSDirectory fsd, final String srcArg)
   static HdfsFileStatus removeDefaultAcl(FSDirectory fsd, final String srcArg)
@@ -92,10 +92,10 @@ class FSDirAclOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsd.resolvePath(pc, src, pathComponents);
     src = fsd.resolvePath(pc, src, pathComponents);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
-      INodesInPath iip = fsd.getINodesInPath4Write(
-          FSDirectory.normalizePath(src), true);
+      iip = fsd.getINodesInPath4Write(FSDirectory.normalizePath(src), true);
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
       INode inode = FSDirectory.resolveLastINode(iip);
       INode inode = FSDirectory.resolveLastINode(iip);
       int snapshotId = iip.getLatestSnapshotId();
       int snapshotId = iip.getLatestSnapshotId();
@@ -107,7 +107,7 @@ class FSDirAclOp {
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static HdfsFileStatus removeAcl(FSDirectory fsd, final String srcArg)
   static HdfsFileStatus removeAcl(FSDirectory fsd, final String srcArg)
@@ -117,16 +117,17 @@ class FSDirAclOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsd.resolvePath(pc, src, pathComponents);
     src = fsd.resolvePath(pc, src, pathComponents);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
-      INodesInPath iip = fsd.getINodesInPath4Write(src);
+      iip = fsd.getINodesInPath4Write(src);
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
       unprotectedRemoveAcl(fsd, iip);
       unprotectedRemoveAcl(fsd, iip);
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
     fsd.getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
     fsd.getEditLog().logSetAcl(src, AclFeature.EMPTY_ENTRY_LIST);
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static HdfsFileStatus setAcl(
   static HdfsFileStatus setAcl(
@@ -137,16 +138,17 @@ class FSDirAclOp {
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     src = fsd.resolvePath(pc, src, pathComponents);
     src = fsd.resolvePath(pc, src, pathComponents);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
-      INodesInPath iip = fsd.getINodesInPath4Write(src);
+      iip = fsd.getINodesInPath4Write(src);
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
       List<AclEntry> newAcl = unprotectedSetAcl(fsd, src, aclSpec);
       List<AclEntry> newAcl = unprotectedSetAcl(fsd, src, aclSpec);
       fsd.getEditLog().logSetAcl(src, newAcl);
       fsd.getEditLog().logSetAcl(src, newAcl);
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static AclStatus getAclStatus(
   static AclStatus getAclStatus(

+ 12 - 8
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java

@@ -50,17 +50,18 @@ public class FSDirAttrOp {
     String src = srcArg;
     String src = srcArg;
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
       src = fsd.resolvePath(pc, src, pathComponents);
       src = fsd.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = fsd.getINodesInPath4Write(src);
+      iip = fsd.getINodesInPath4Write(src);
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
       unprotectedSetPermission(fsd, src, permission);
       unprotectedSetPermission(fsd, src, permission);
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
     fsd.getEditLog().logSetPermissions(src, permission);
     fsd.getEditLog().logSetPermissions(src, permission);
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static HdfsFileStatus setOwner(
   static HdfsFileStatus setOwner(
@@ -68,10 +69,11 @@ public class FSDirAttrOp {
       throws IOException {
       throws IOException {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
       src = fsd.resolvePath(pc, src, pathComponents);
       src = fsd.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = fsd.getINodesInPath4Write(src);
+      iip = fsd.getINodesInPath4Write(src);
       fsd.checkOwner(pc, iip);
       fsd.checkOwner(pc, iip);
       if (!pc.isSuperUser()) {
       if (!pc.isSuperUser()) {
         if (username != null && !pc.getUser().equals(username)) {
         if (username != null && !pc.getUser().equals(username)) {
@@ -86,7 +88,7 @@ public class FSDirAttrOp {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
     fsd.getEditLog().logSetOwner(src, username, group);
     fsd.getEditLog().logSetOwner(src, username, group);
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static HdfsFileStatus setTimes(
   static HdfsFileStatus setTimes(
@@ -102,10 +104,11 @@ public class FSDirAttrOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
 
 
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
       src = fsd.resolvePath(pc, src, pathComponents);
       src = fsd.resolvePath(pc, src, pathComponents);
-      final INodesInPath iip = fsd.getINodesInPath4Write(src);
+      iip = fsd.getINodesInPath4Write(src);
       // Write access is required to set access and modification times
       // Write access is required to set access and modification times
       if (fsd.isPermissionEnabled()) {
       if (fsd.isPermissionEnabled()) {
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
@@ -123,7 +126,7 @@ public class FSDirAttrOp {
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static boolean setReplication(
   static boolean setReplication(
@@ -165,10 +168,11 @@ public class FSDirAttrOp {
     }
     }
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
       src = FSDirectory.resolvePath(src, pathComponents, fsd);
       src = FSDirectory.resolvePath(src, pathComponents, fsd);
-      final INodesInPath iip = fsd.getINodesInPath4Write(src);
+      iip = fsd.getINodesInPath4Write(src);
 
 
       if (fsd.isPermissionEnabled()) {
       if (fsd.isPermissionEnabled()) {
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
         fsd.checkPathAccess(pc, iip, FsAction.WRITE);
@@ -185,7 +189,7 @@ public class FSDirAttrOp {
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static BlockStoragePolicy[] getStoragePolicies(BlockManager bm)
   static BlockStoragePolicy[] getStoragePolicies(BlockManager bm)

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

@@ -168,7 +168,7 @@ class FSDirConcatOp {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
     fsd.getEditLog().logConcat(target, srcs, timestamp, logRetryCache);
     fsd.getEditLog().logConcat(target, srcs, timestamp, logRetryCache);
-    return fsd.getAuditFileInfo(target, false);
+    return fsd.getAuditFileInfo(trgIip);
   }
   }
 
 
   /**
   /**

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

@@ -42,7 +42,6 @@ class FSDirMkdirOp {
       FSNamesystem fsn, String src, PermissionStatus permissions,
       FSNamesystem fsn, String src, PermissionStatus permissions,
       boolean createParent) throws IOException {
       boolean createParent) throws IOException {
     FSDirectory fsd = fsn.getFSDirectory();
     FSDirectory fsd = fsn.getFSDirectory();
-    final String srcArg = src;
     if(NameNode.stateChangeLog.isDebugEnabled()) {
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
       NameNode.stateChangeLog.debug("DIR* NameSystem.mkdirs: " + src);
     }
     }
@@ -70,12 +69,12 @@ class FSDirMkdirOp {
       // heuristic because the mkdirs() operation might need to
       // heuristic because the mkdirs() operation might need to
       // create multiple inodes.
       // create multiple inodes.
       fsn.checkFsObjectLimit();
       fsn.checkFsObjectLimit();
-
-      if (mkdirsRecursively(fsd, iip, permissions, false, now()) == null) {
+      iip = mkdirsRecursively(fsd, iip, permissions, false, now());
+      if (iip == null) {
         throw new IOException("Failed to create directory: " + src);
         throw new IOException("Failed to create directory: " + src);
       }
       }
     }
     }
-    return fsd.getAuditFileInfo(srcArg, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static INode unprotectedMkdir(
   static INode unprotectedMkdir(

+ 9 - 6
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java

@@ -70,7 +70,8 @@ class FSDirRenameOp {
     @SuppressWarnings("deprecation")
     @SuppressWarnings("deprecation")
     final boolean status = renameTo(fsd, pc, src, dst, logRetryCache);
     final boolean status = renameTo(fsd, pc, src, dst, logRetryCache);
     if (status) {
     if (status) {
-      resultingStat = fsd.getAuditFileInfo(dst, false);
+      INodesInPath dstIIP = fsd.getINodesInPath(dst, false);
+      resultingStat = fsd.getAuditFileInfo(dstIIP);
     }
     }
     return new RenameOldResult(status, resultingStat);
     return new RenameOldResult(status, resultingStat);
   }
   }
@@ -122,6 +123,7 @@ class FSDirRenameOp {
    * <br>
    * <br>
    */
    */
   @Deprecated
   @Deprecated
+  @SuppressWarnings("deprecation")
   static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst,
   static boolean unprotectedRenameTo(FSDirectory fsd, String src, String dst,
       long timestamp) throws IOException {
       long timestamp) throws IOException {
     if (fsd.isDir(dst)) {
     if (fsd.isDir(dst)) {
@@ -246,10 +248,11 @@ class FSDirRenameOp {
     src = fsd.resolvePath(pc, src, srcComponents);
     src = fsd.resolvePath(pc, src, srcComponents);
     dst = fsd.resolvePath(pc, dst, dstComponents);
     dst = fsd.resolvePath(pc, dst, dstComponents);
     renameTo(fsd, pc, src, dst, collectedBlocks, logRetryCache, options);
     renameTo(fsd, pc, src, dst, collectedBlocks, logRetryCache, options);
-    HdfsFileStatus resultingStat = fsd.getAuditFileInfo(dst, false);
+    INodesInPath dstIIP = fsd.getINodesInPath(dst, false);
+    HdfsFileStatus resultingStat = fsd.getAuditFileInfo(dstIIP);
 
 
-    return new AbstractMap.SimpleImmutableEntry<BlocksMapUpdateInfo,
-        HdfsFileStatus>(collectedBlocks, resultingStat);
+    return new AbstractMap.SimpleImmutableEntry<>(
+        collectedBlocks, resultingStat);
   }
   }
 
 
   /**
   /**
@@ -357,7 +360,7 @@ class FSDirRenameOp {
 
 
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
     final INode dstInode = dstIIP.getLastINode();
     final INode dstInode = dstIIP.getLastINode();
-    List<INodeDirectory> snapshottableDirs = new ArrayList<INodeDirectory>();
+    List<INodeDirectory> snapshottableDirs = new ArrayList<>();
     if (dstInode != null) { // Destination exists
     if (dstInode != null) { // Destination exists
       validateOverwrite(src, dst, overwrite, srcInode, dstInode);
       validateOverwrite(src, dst, overwrite, srcInode, dstInode);
       FSDirSnapshotOp.checkSnapshot(dstInode, snapshottableDirs);
       FSDirSnapshotOp.checkSnapshot(dstInode, snapshottableDirs);
@@ -419,7 +422,7 @@ class FSDirRenameOp {
         if (removedDst != null) {
         if (removedDst != null) {
           undoRemoveDst = false;
           undoRemoveDst = false;
           if (removedNum > 0) {
           if (removedNum > 0) {
-            List<INode> removedINodes = new ChunkedArrayList<INode>();
+            List<INode> removedINodes = new ChunkedArrayList<>();
             if (!removedDst.isInLatestSnapshot(tx.dstIIP.getLatestSnapshotId())) {
             if (!removedDst.isInLatestSnapshot(tx.dstIIP.getLatestSnapshotId())) {
               removedDst.destroyAndCollectBlocks(collectedBlocks,
               removedDst.destroyAndCollectBlocks(collectedBlocks,
                   removedINodes);
                   removedINodes);

+ 28 - 11
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java

@@ -264,29 +264,46 @@ class FSDirStatAndListingOp {
   /** Get the file info for a specific file.
   /** Get the file info for a specific file.
    * @param fsd FSDirectory
    * @param fsd FSDirectory
    * @param src The string representation of the path to the file
    * @param src The string representation of the path to the file
-   * @param resolveLink whether to throw UnresolvedLinkException
    * @param isRawPath true if a /.reserved/raw pathname was passed by the user
    * @param isRawPath true if a /.reserved/raw pathname was passed by the user
    * @param includeStoragePolicy whether to include storage policy
    * @param includeStoragePolicy whether to include storage policy
    * @return object containing information regarding the file
    * @return object containing information regarding the file
    *         or null if file not found
    *         or null if file not found
    */
    */
+  static HdfsFileStatus getFileInfo(
+      FSDirectory fsd, INodesInPath src, boolean isRawPath,
+      boolean includeStoragePolicy)
+      throws IOException {
+    fsd.readLock();
+    try {
+      final INode i = src.getLastINode();
+      byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
+          i.getStoragePolicyID() : BlockStoragePolicySuite.ID_UNSPECIFIED;
+      return i == null ? null : createFileStatus(
+          fsd, HdfsFileStatus.EMPTY_NAME, i, policyId,
+          src.getPathSnapshotId(), isRawPath, src);
+    } finally {
+      fsd.readUnlock();
+    }
+  }
+
   static HdfsFileStatus getFileInfo(
   static HdfsFileStatus getFileInfo(
       FSDirectory fsd, String src, boolean resolveLink, boolean isRawPath,
       FSDirectory fsd, String src, boolean resolveLink, boolean isRawPath,
       boolean includeStoragePolicy)
       boolean includeStoragePolicy)
     throws IOException {
     throws IOException {
     String srcs = FSDirectory.normalizePath(src);
     String srcs = FSDirectory.normalizePath(src);
+    if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
+      if (fsd.getINode4DotSnapshot(srcs) != null) {
+        return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null,
+            HdfsFileStatus.EMPTY_NAME, -1L, 0, null,
+            BlockStoragePolicySuite.ID_UNSPECIFIED);
+      }
+      return null;
+    }
+
     fsd.readLock();
     fsd.readLock();
     try {
     try {
-      if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
-        return getFileInfo4DotSnapshot(fsd, srcs);
-      }
-      final INodesInPath inodesInPath = fsd.getINodesInPath(srcs, resolveLink);
-      final INode i = inodesInPath.getLastINode();
-      byte policyId = includeStoragePolicy && i != null && !i.isSymlink() ?
-          i.getStoragePolicyID() : BlockStoragePolicySuite.ID_UNSPECIFIED;
-      return i == null ? null : createFileStatus(fsd,
-          HdfsFileStatus.EMPTY_NAME, i, policyId,
-          inodesInPath.getPathSnapshotId(), isRawPath, inodesInPath);
+      final INodesInPath iip = fsd.getINodesInPath(srcs, resolveLink);
+      return getFileInfo(fsd, iip, isRawPath, includeStoragePolicy);
     } finally {
     } finally {
       fsd.readUnlock();
       fsd.readUnlock();
     }
     }

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirSymlinkOp.java

@@ -52,10 +52,11 @@ class FSDirSymlinkOp {
 
 
     FSPermissionChecker pc = fsn.getPermissionChecker();
     FSPermissionChecker pc = fsn.getPermissionChecker();
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(link);
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(link);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
       link = fsd.resolvePath(pc, link, pathComponents);
       link = fsd.resolvePath(pc, link, pathComponents);
-      final INodesInPath iip = fsd.getINodesInPath4Write(link, false);
+      iip = fsd.getINodesInPath4Write(link, false);
       if (!createParent) {
       if (!createParent) {
         fsd.verifyParentDir(iip, link);
         fsd.verifyParentDir(iip, link);
       }
       }
@@ -76,7 +77,7 @@ class FSDirSymlinkOp {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
     NameNode.getNameNodeMetrics().incrCreateSymlinkOps();
     NameNode.getNameNodeMetrics().incrCreateSymlinkOps();
-    return fsd.getAuditFileInfo(link, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static INodeSymlink unprotectedAddSymlink(
   static INodeSymlink unprotectedAddSymlink(

+ 11 - 9
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java

@@ -66,21 +66,21 @@ class FSDirXAttrOp {
     FSPermissionChecker pc = fsd.getPermissionChecker();
     FSPermissionChecker pc = fsd.getPermissionChecker();
     XAttrPermissionFilter.checkPermissionForApi(
     XAttrPermissionFilter.checkPermissionForApi(
         pc, xAttr, FSDirectory.isReservedRawName(src));
         pc, xAttr, FSDirectory.isReservedRawName(src));
-    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
-        src);
+    byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src);
     src = fsd.resolvePath(pc, src, pathComponents);
     src = fsd.resolvePath(pc, src, pathComponents);
-    final INodesInPath iip = fsd.getINodesInPath4Write(src);
-    checkXAttrChangeAccess(fsd, iip, xAttr, pc);
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     xAttrs.add(xAttr);
     xAttrs.add(xAttr);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
+      iip = fsd.getINodesInPath4Write(src);
+      checkXAttrChangeAccess(fsd, iip, xAttr, pc);
       unprotectedSetXAttrs(fsd, src, xAttrs, flag);
       unprotectedSetXAttrs(fsd, src, xAttrs, flag);
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
     fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
     fsd.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static List<XAttr> getXAttrs(FSDirectory fsd, final String srcArg,
   static List<XAttr> getXAttrs(FSDirectory fsd, final String srcArg,
@@ -164,14 +164,16 @@ class FSDirXAttrOp {
         pc, xAttr, FSDirectory.isReservedRawName(src));
         pc, xAttr, FSDirectory.isReservedRawName(src));
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
     byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(
         src);
         src);
-    src = fsd.resolvePath(pc, src, pathComponents);
-    final INodesInPath iip = fsd.getINodesInPath4Write(src);
-    checkXAttrChangeAccess(fsd, iip, xAttr, pc);
 
 
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
     xAttrs.add(xAttr);
     xAttrs.add(xAttr);
+    INodesInPath iip;
     fsd.writeLock();
     fsd.writeLock();
     try {
     try {
+      src = fsd.resolvePath(pc, src, pathComponents);
+      iip = fsd.getINodesInPath4Write(src);
+      checkXAttrChangeAccess(fsd, iip, xAttr, pc);
+
       List<XAttr> removedXAttrs = unprotectedRemoveXAttrs(fsd, src, xAttrs);
       List<XAttr> removedXAttrs = unprotectedRemoveXAttrs(fsd, src, xAttrs);
       if (removedXAttrs != null && !removedXAttrs.isEmpty()) {
       if (removedXAttrs != null && !removedXAttrs.isEmpty()) {
         fsd.getEditLog().logRemoveXAttrs(src, removedXAttrs, logRetryCache);
         fsd.getEditLog().logRemoveXAttrs(src, removedXAttrs, logRetryCache);
@@ -182,7 +184,7 @@ class FSDirXAttrOp {
     } finally {
     } finally {
       fsd.writeUnlock();
       fsd.writeUnlock();
     }
     }
-    return fsd.getAuditFileInfo(src, false);
+    return fsd.getAuditFileInfo(iip);
   }
   }
 
 
   static List<XAttr> unprotectedRemoveXAttrs(
   static List<XAttr> unprotectedRemoveXAttrs(

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

@@ -1647,11 +1647,10 @@ public class FSDirectory implements Closeable {
     }
     }
   }
   }
 
 
-  HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink)
-    throws IOException {
+  HdfsFileStatus getAuditFileInfo(INodesInPath iip)
+      throws IOException {
     return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation())
     return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation())
-      ? FSDirStatAndListingOp.getFileInfo(this, path, resolveSymlink, false,
-        false) : null;
+        ? FSDirStatAndListingOp.getFileInfo(this, iip, false, false) : null;
   }
   }
 
 
   /**
   /**

+ 3 - 7
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java

@@ -337,11 +337,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return !isDefaultAuditLogger || auditLog.isInfoEnabled();
     return !isDefaultAuditLogger || auditLog.isInfoEnabled();
   }
   }
 
 
-  private HdfsFileStatus getAuditFileInfo(String path, boolean resolveSymlink)
-      throws IOException {
-    return dir.getAuditFileInfo(path, resolveSymlink);
-  }
-  
   private void logAuditEvent(boolean succeeded, String cmd, String src)
   private void logAuditEvent(boolean succeeded, String cmd, String src)
       throws IOException {
       throws IOException {
     logAuditEvent(succeeded, cmd, src, null, null);
     logAuditEvent(succeeded, cmd, src, null, null);
@@ -7669,7 +7664,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
       xAttrs.add(ezXAttr);
       xAttrs.add(ezXAttr);
       getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
       getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
-      resultingStat = getAuditFileInfo(src, false);
+      final INodesInPath iip = dir.getINodesInPath4Write(src, false);
+      resultingStat = dir.getAuditFileInfo(iip);
     } finally {
     } finally {
       writeUnlock();
       writeUnlock();
     }
     }
@@ -7703,7 +7699,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         dir.checkPathAccess(pc, iip, FsAction.READ);
         dir.checkPathAccess(pc, iip, FsAction.READ);
       }
       }
       final EncryptionZone ret = dir.getEZForPath(iip);
       final EncryptionZone ret = dir.getEZForPath(iip);
-      resultingStat = getAuditFileInfo(src, false);
+      resultingStat = dir.getAuditFileInfo(iip);
       success = true;
       success = true;
       return ret;
       return ret;
     } finally {
     } finally {