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 su...@apache.org on 2016/10/03 22:14:10 UTC

[11/50] [abbrv] hadoop git commit: HDFS-10851. FSDirStatAndListingOp: stop passing path as string. Contributed by Daryn Sharp.

HDFS-10851. FSDirStatAndListingOp: stop passing path as string. Contributed by Daryn Sharp.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a0730aa5
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a0730aa5
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a0730aa5

Branch: refs/heads/YARN-2915
Commit: a0730aa5ced7666a8c92f9fb830b615f5f9f477a
Parents: 0670149
Author: Kihwal Lee <ki...@apache.org>
Authored: Fri Sep 30 13:03:24 2016 -0500
Committer: Kihwal Lee <ki...@apache.org>
Committed: Fri Sep 30 13:03:24 2016 -0500

----------------------------------------------------------------------
 .../hadoop/hdfs/server/namenode/FSDirAclOp.java |   4 +-
 .../server/namenode/FSDirStatAndListingOp.java  | 256 +++++++------------
 .../hdfs/server/namenode/FSDirXAttrOp.java      |   6 +-
 .../hdfs/server/namenode/FSDirectory.java       |  17 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  18 +-
 .../server/namenode/INodeAttributeProvider.java |  10 +-
 .../hdfs/server/namenode/INodesInPath.java      |   2 +-
 .../server/namenode/TestSnapshotPathINodes.java |   6 +
 8 files changed, 126 insertions(+), 193 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
index 2153f02..afafd78 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAclOp.java
@@ -152,7 +152,6 @@ class FSDirAclOp {
     fsd.readLock();
     try {
       INodesInPath iip = fsd.resolvePath(pc, src);
-      src = iip.getPath();
       // There is no real inode for the path ending in ".snapshot", so return a
       // non-null, unpopulated AclStatus.  This is similar to getFileInfo.
       if (iip.isDotSnapshotDir() && fsd.getINode4DotSnapshot(iip) != null) {
@@ -163,8 +162,7 @@ class FSDirAclOp {
       }
       INode inode = FSDirectory.resolveLastINode(iip);
       int snapshotId = iip.getPathSnapshotId();
-      List<AclEntry> acl = AclStorage.readINodeAcl(fsd.getAttributes(src,
-              inode.getLocalNameBytes(), inode, snapshotId));
+      List<AclEntry> acl = AclStorage.readINodeAcl(fsd.getAttributes(iip));
       FsPermission fsPermission = inode.getFsPermission(snapshotId);
       return new AclStatus.Builder()
           .owner(inode.getUserName()).group(inode.getGroupName())

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
index 5072d68..4876fb1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java
@@ -53,15 +53,12 @@ import static org.apache.hadoop.util.Time.now;
 class FSDirStatAndListingOp {
   static DirectoryListing getListingInt(FSDirectory fsd, final String srcArg,
       byte[] startAfter, boolean needLocation) throws IOException {
-    String src = null;
-
     final INodesInPath iip;
     if (fsd.isPermissionEnabled()) {
       FSPermissionChecker pc = fsd.getPermissionChecker();
       iip = fsd.resolvePath(pc, srcArg);
-      src = iip.getPath();
     } else {
-      src = FSDirectory.resolvePath(srcArg, fsd);
+      String src = FSDirectory.resolvePath(srcArg, fsd);
       iip = fsd.getINodesInPath(src, true);
     }
 
@@ -92,7 +89,7 @@ class FSDirStatAndListingOp {
       }
       isSuperUser = pc.isSuperUser();
     }
-    return getListing(fsd, iip, src, startAfter, needLocation, isSuperUser);
+    return getListing(fsd, iip, startAfter, needLocation, isSuperUser);
   }
 
   /**
@@ -161,7 +158,6 @@ class FSDirStatAndListingOp {
         "Negative length is not supported. File: " + src);
     CacheManager cm = fsd.getFSNamesystem().getCacheManager();
     BlockManager bm = fsd.getBlockManager();
-    boolean isReservedName = FSDirectory.isReservedRawName(src);
     fsd.readLock();
     try {
       final INodesInPath iip = fsd.resolvePath(pc, src);
@@ -184,7 +180,7 @@ class FSDirStatAndListingOp {
         isUc = false;
       }
 
-      final FileEncryptionInfo feInfo = isReservedName ? null
+      final FileEncryptionInfo feInfo = iip.isRaw() ? null
           : FSDirEncryptionZoneOp.getFileEncryptionInfo(fsd, inode,
           iip.getPathSnapshotId(), iip);
       final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.
@@ -225,42 +221,39 @@ class FSDirStatAndListingOp {
    * @param fsd FSDirectory
    * @param iip the INodesInPath instance containing all the INodes along the
    *            path
-   * @param src the directory name
    * @param startAfter the name to start listing after
    * @param needLocation if block locations are returned
+   * @param includeStoragePolicy if storage policy is returned
    * @return a partial listing starting after startAfter
    */
   private static DirectoryListing getListing(FSDirectory fsd, INodesInPath iip,
-      String src, byte[] startAfter, boolean needLocation, boolean isSuperUser)
+      byte[] startAfter, boolean needLocation, boolean includeStoragePolicy)
       throws IOException {
-    String srcs = FSDirectory.normalizePath(src);
-    if (FSDirectory.isExactReservedName(srcs)) {
+    if (FSDirectory.isExactReservedName(iip.getPathComponents())) {
       return getReservedListing(fsd);
     }
 
     fsd.readLock();
     try {
-      if (srcs.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR)) {
-        return getSnapshotsListing(fsd, srcs, startAfter);
+      if (iip.isDotSnapshotDir()) {
+        return getSnapshotsListing(fsd, iip, startAfter);
       }
       final int snapshot = iip.getPathSnapshotId();
       final INode targetNode = iip.getLastINode();
-      if (targetNode == null)
+      if (targetNode == null) {
         return null;
-      byte parentStoragePolicy = isSuperUser ?
-          targetNode.getStoragePolicyID() : HdfsConstants
-          .BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      }
+
+      byte parentStoragePolicy = includeStoragePolicy
+          ? targetNode.getStoragePolicyID()
+          : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
 
       if (!targetNode.isDirectory()) {
         // return the file's status. note that the iip already includes the
         // target INode
-        INodeAttributes nodeAttrs = getINodeAttributes(
-            fsd, src, HdfsFileStatus.EMPTY_NAME, targetNode,
-            snapshot);
         return new DirectoryListing(
             new HdfsFileStatus[]{ createFileStatus(
-                fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                needLocation, parentStoragePolicy, iip)
+                fsd, iip, null, parentStoragePolicy, needLocation)
             }, 0);
       }
 
@@ -274,20 +267,15 @@ class FSDirStatAndListingOp {
       int listingCnt = 0;
       HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
       for (int i = 0; i < numOfListing && locationBudget > 0; i++) {
-        INode cur = contents.get(startChild+i);
-        byte curPolicy = isSuperUser && !cur.isSymlink()?
-            cur.getLocalStoragePolicyID():
-            HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-        INodeAttributes nodeAttrs = getINodeAttributes(
-            fsd, src, cur.getLocalNameBytes(), cur,
-            snapshot);
-        final INodesInPath iipWithChild = INodesInPath.append(iip, cur,
-            cur.getLocalNameBytes());
-        listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), nodeAttrs,
-            needLocation, getStoragePolicyID(curPolicy, parentStoragePolicy),
-            iipWithChild);
+        INode child = contents.get(startChild+i);
+        byte childStoragePolicy = (includeStoragePolicy && !child.isSymlink())
+            ? getStoragePolicyID(child.getLocalStoragePolicyID(),
+                                 parentStoragePolicy)
+            : parentStoragePolicy;
+        listing[i] =
+            createFileStatus(fsd, iip, child, childStoragePolicy, needLocation);
         listingCnt++;
-        if (needLocation) {
+        if (listing[i] instanceof HdfsLocatedFileStatus) {
             // Once we  hit lsLimit locations, stop.
             // This helps to prevent excessively large response payloads.
             // Approximate #locations with locatedBlockCount() * repl_factor
@@ -312,17 +300,16 @@ class FSDirStatAndListingOp {
    * Get a listing of all the snapshots of a snapshottable directory
    */
   private static DirectoryListing getSnapshotsListing(
-      FSDirectory fsd, String src, byte[] startAfter)
+      FSDirectory fsd, INodesInPath iip, byte[] startAfter)
       throws IOException {
     Preconditions.checkState(fsd.hasReadLock());
-    Preconditions.checkArgument(
-        src.endsWith(HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR),
-        "%s does not end with %s", src, HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
-
-    final String dirPath = FSDirectory.normalizePath(src.substring(0,
-        src.length() - HdfsConstants.DOT_SNAPSHOT_DIR.length()));
-
-    final INode node = fsd.getINode(dirPath);
+    Preconditions.checkArgument(iip.isDotSnapshotDir(),
+        "%s does not end with %s",
+        iip.getPath(), HdfsConstants.SEPARATOR_DOT_SNAPSHOT_DIR);
+    // drop off the null .snapshot component
+    iip = iip.getParentINodesInPath();
+    final String dirPath = iip.getPath();
+    final INode node = iip.getLastINode();
     final INodeDirectory dirNode = INodeDirectory.valueOf(node, dirPath);
     final DirectorySnapshottableFeature sf = dirNode.getDirectorySnapshottableFeature();
     if (sf == null) {
@@ -336,13 +323,8 @@ class FSDirStatAndListingOp {
     final HdfsFileStatus listing[] = new HdfsFileStatus[numOfListing];
     for (int i = 0; i < numOfListing; i++) {
       Snapshot.Root sRoot = snapshots.get(i + skipSize).getRoot();
-      INodeAttributes nodeAttrs = getINodeAttributes(
-          fsd, src, sRoot.getLocalNameBytes(),
-          node, Snapshot.CURRENT_STATE_ID);
-      listing[i] = createFileStatus(
-          fsd, sRoot.getLocalNameBytes(), nodeAttrs,
-          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-          INodesInPath.fromINode(sRoot));
+      listing[i] = createFileStatus(fsd, iip, sRoot,
+          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false);
     }
     return new DirectoryListing(
         listing, snapshots.size() - skipSize - numOfListing);
@@ -360,7 +342,6 @@ class FSDirStatAndListingOp {
   /** Get the file info for a specific file.
    * @param fsd FSDirectory
    * @param iip The path to the file, the file is included
-   * @param isRawPath true if a /.reserved/raw pathname was passed by the user
    * @param includeStoragePolicy whether to include storage policy
    * @return object containing information regarding the file
    *         or null if file not found
@@ -373,15 +354,10 @@ class FSDirStatAndListingOp {
       if (node == null) {
         return null;
       }
-
-      byte policyId = includeStoragePolicy && !node.isSymlink() ?
-          node.getStoragePolicyID() :
-          HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
-      INodeAttributes nodeAttrs = getINodeAttributes(fsd, iip.getPath(),
-                                                     HdfsFileStatus.EMPTY_NAME,
-                                                     node, iip.getPathSnapshotId());
-      return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs,
-                              policyId, iip);
+      byte policy = (includeStoragePolicy && !node.isSymlink())
+          ? node.getStoragePolicyID()
+          : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED;
+      return createFileStatus(fsd, iip, null, policy, false);
     } finally {
       fsd.readUnlock();
     }
@@ -408,48 +384,41 @@ class FSDirStatAndListingOp {
   }
 
   /**
-   * create an hdfs file status from an inode
+   * create a hdfs file status from an iip.
+   * @param fsd FSDirectory
+   * @param iip The INodesInPath containing the INodeFile and its ancestors
+   * @return HdfsFileStatus without locations or storage policy
+   */
+  static HdfsFileStatus createFileStatusForEditLog(
+      FSDirectory fsd, INodesInPath iip) throws IOException {
+    return createFileStatus(fsd, iip,
+        null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, false);
+  }
+
+  /**
+   * create a hdfs file status from an iip.
    *
    * @param fsd FSDirectory
-   * @param path the local name
+   * @param iip The INodesInPath containing the INodeFile and its ancestors.
+   * @param child for a directory listing of the iip, else null
+   * @param storagePolicy for the path or closest ancestor
    * @param needLocation if block locations need to be included or not
-   * @param isRawPath true if this is being called on behalf of a path in
-   *                  /.reserved/raw
-   * @param iip the INodesInPath containing the target INode and its ancestors
+   * @param includeStoragePolicy if storage policy should be returned
    * @return a file status
    * @throws java.io.IOException if any error occurs
    */
   private static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      boolean needLocation, byte storagePolicy, INodesInPath iip)
-      throws IOException {
-    if (needLocation) {
-      return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
-    } else {
-      return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
+      FSDirectory fsd, INodesInPath iip, INode child, byte storagePolicy,
+      boolean needLocation) throws IOException {
+    assert fsd.hasReadLock();
+    // only directory listing sets the status name.
+    byte[] name = HdfsFileStatus.EMPTY_NAME;
+    if (child != null) {
+      name = child.getLocalNameBytes();
+      // have to do this for EC and EZ lookups...
+      iip = INodesInPath.append(iip, child, name);
     }
-  }
 
-  /**
-   * Create FileStatus for an given INodeFile.
-   * @param iip The INodesInPath containing the INodeFile and its ancestors
-   */
-  static HdfsFileStatus createFileStatusForEditLog(
-      FSDirectory fsd, String fullPath, byte[] path,
-      byte storagePolicy, int snapshot, boolean isRawPath,
-      INodesInPath iip) throws IOException {
-    INodeAttributes nodeAttrs = getINodeAttributes(
-        fsd, fullPath, path, iip.getLastINode(), snapshot);
-    return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip);
-  }
-
-  /**
-   * create file status for a given INode
-   * @param iip the INodesInPath containing the target INode and its ancestors
-   */
-  static HdfsFileStatus createFileStatus(
-      FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      byte storagePolicy, INodesInPath iip) throws IOException {
     long size = 0;     // length is zero for directories
     short replication = 0;
     long blocksize = 0;
@@ -457,6 +426,7 @@ class FSDirStatAndListingOp {
     final INode node = iip.getLastINode();
     final int snapshot = iip.getPathSnapshotId();
     final boolean isRawPath = iip.isRaw();
+    LocatedBlocks loc = null;
 
     final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
         .getFileEncryptionInfo(fsd, node, snapshot, iip);
@@ -471,6 +441,18 @@ class FSDirStatAndListingOp {
       blocksize = fileNode.getPreferredBlockSize();
       isEncrypted = (feInfo != null)
           || (isRawPath && FSDirEncryptionZoneOp.isInAnEZ(fsd, iip));
+      if (needLocation) {
+        final boolean inSnapshot = snapshot != Snapshot.CURRENT_STATE_ID;
+        final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
+        final long fileSize = !inSnapshot && isUc
+            ? fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
+        loc = fsd.getBlockManager().createLocatedBlocks(
+            fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
+            inSnapshot, feInfo, ecPolicy);
+        if (loc == null) {
+          loc = new LocatedBlocks();
+        }
+      }
     } else {
       isEncrypted = FSDirEncryptionZoneOp.isInAnEZ(fsd, iip);
     }
@@ -478,7 +460,8 @@ class FSDirStatAndListingOp {
     int childrenNum = node.isDirectory() ?
         node.asDirectory().getChildrenNum(snapshot) : 0;
 
-    return new HdfsFileStatus(
+    INodeAttributes nodeAttrs = fsd.getAttributes(iip);
+    HdfsFileStatus status = createFileStatus(
         size,
         node.isDirectory(),
         replication,
@@ -489,73 +472,13 @@ class FSDirStatAndListingOp {
         nodeAttrs.getUserName(),
         nodeAttrs.getGroupName(),
         node.isSymlink() ? node.asSymlink().getSymlink() : null,
-        path,
+        name,
         node.getId(),
         childrenNum,
         feInfo,
         storagePolicy,
-        ecPolicy);
-  }
-
-  private static INodeAttributes getINodeAttributes(
-      FSDirectory fsd, String fullPath, byte[] path, INode node, int snapshot) {
-    return fsd.getAttributes(fullPath, path, node, snapshot);
-  }
-
-  /**
-   * Create FileStatus with location info by file INode
-   * @param iip the INodesInPath containing the target INode and its ancestors
-   */
-  private static HdfsFileStatus createLocatedFileStatus(
-      FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs,
-      byte storagePolicy, INodesInPath iip) throws IOException {
-    assert fsd.hasReadLock();
-    long size = 0; // length is zero for directories
-    short replication = 0;
-    long blocksize = 0;
-    LocatedBlocks loc = null;
-    final boolean isEncrypted;
-    final INode node = iip.getLastINode();
-    final int snapshot = iip.getPathSnapshotId();
-    final boolean isRawPath = iip.isRaw();
-
-    final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp
-        .getFileEncryptionInfo(fsd, node, snapshot, iip);
-    final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(
-        fsd.getFSNamesystem(), iip);
-    if (node.isFile()) {
-      final INodeFile fileNode = node.asFile();
-      size = fileNode.computeFileSize(snapshot);
-      replication = fileNode.getFileReplication(snapshot);
-      blocksize = fileNode.getPreferredBlockSize();
-
-      final boolean inSnapshot = snapshot != Snapshot.CURRENT_STATE_ID;
-      final boolean isUc = !inSnapshot && fileNode.isUnderConstruction();
-      final long fileSize = !inSnapshot && isUc ?
-          fileNode.computeFileSizeNotIncludingLastUcBlock() : size;
-
-      loc = fsd.getBlockManager().createLocatedBlocks(
-          fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false,
-          inSnapshot, feInfo, ecPolicy);
-      if (loc == null) {
-        loc = new LocatedBlocks();
-      }
-      isEncrypted = (feInfo != null)
-          || (isRawPath && FSDirEncryptionZoneOp.isInAnEZ(fsd, iip));
-    } else {
-      isEncrypted = FSDirEncryptionZoneOp.isInAnEZ(fsd, iip);
-    }
-    int childrenNum = node.isDirectory() ?
-        node.asDirectory().getChildrenNum(snapshot) : 0;
-
-    HdfsLocatedFileStatus status =
-        new HdfsLocatedFileStatus(size, node.isDirectory(), replication,
-          blocksize, node.getModificationTime(snapshot),
-          node.getAccessTime(snapshot),
-          getPermissionForFileStatus(nodeAttrs, isEncrypted),
-          nodeAttrs.getUserName(), nodeAttrs.getGroupName(),
-          node.isSymlink() ? node.asSymlink().getSymlink() : null, path,
-          node.getId(), loc, childrenNum, feInfo, storagePolicy, ecPolicy);
+        ecPolicy,
+        loc);
     // Set caching information for the located blocks.
     if (loc != null) {
       CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager();
@@ -566,6 +489,23 @@ class FSDirStatAndListingOp {
     return status;
   }
 
+  private static HdfsFileStatus createFileStatus(long length, boolean isdir,
+      int replication, long blocksize, long mtime,
+      long atime, FsPermission permission, String owner, String group,
+      byte[] symlink, byte[] path, long fileId, int childrenNum,
+      FileEncryptionInfo feInfo, byte storagePolicy,
+      ErasureCodingPolicy ecPolicy, LocatedBlocks locations) {
+    if (locations == null) {
+      return new HdfsFileStatus(length, isdir, replication, blocksize,
+          mtime, atime, permission, owner, group, symlink, path, fileId,
+          childrenNum, feInfo, storagePolicy, ecPolicy);
+    } else {
+      return new HdfsLocatedFileStatus(length, isdir, replication, blocksize,
+          mtime, atime, permission, owner, group, symlink, path, fileId,
+          locations, childrenNum, feInfo, storagePolicy, ecPolicy);
+    }
+  }
+
   /**
    * Returns an inode's FsPermission for use in an outbound FileStatus.  If the
    * inode has an ACL or is for an encrypted file/dir, then this method will

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
index 746fdb7..08016c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirXAttrOp.java
@@ -429,11 +429,7 @@ class FSDirXAttrOp {
       throws IOException {
     fsd.readLock();
     try {
-      String src = iip.getPath();
-      INode inode = FSDirectory.resolveLastINode(iip);
-      int snapshotId = iip.getPathSnapshotId();
-      return XAttrStorage.readINodeXAttrs(fsd.getAttributes(src,
-              inode.getLocalNameBytes(), inode, snapshotId));
+      return XAttrStorage.readINodeXAttrs(fsd.getAttributes(iip));
     } finally {
       fsd.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 2c7a268..7db2106 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -1807,14 +1807,19 @@ public class FSDirectory implements Closeable {
     inodeId.setCurrentValue(newValue);
   }
 
-  INodeAttributes getAttributes(String fullPath, byte[] path,
-      INode node, int snapshot) {
+  INodeAttributes getAttributes(INodesInPath iip)
+      throws FileNotFoundException {
+    INode node = FSDirectory.resolveLastINode(iip);
+    int snapshot = iip.getPathSnapshotId();
     INodeAttributes nodeAttrs = node.getSnapshotINode(snapshot);
     if (attributeProvider != null) {
-      fullPath = fullPath
-          + (fullPath.endsWith(Path.SEPARATOR) ? "" : Path.SEPARATOR)
-          + DFSUtil.bytes2String(path);
-      nodeAttrs = attributeProvider.getAttributes(fullPath, nodeAttrs);
+      // permission checking sends the full components array including the
+      // first empty component for the root.  however file status
+      // related calls are expected to strip out the root component according
+      // to TestINodeAttributeProvider.
+      byte[][] components = iip.getPathComponents();
+      components = Arrays.copyOfRange(components, 1, components.length);
+      nodeAttrs = attributeProvider.getAttributes(components, nodeAttrs);
     }
     return nodeAttrs;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 25f5a4f..ee6206a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -383,10 +383,8 @@ public class FSEditLogLoader {
 
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
-              fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME,
-              HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, Snapshot.CURRENT_STATE_ID,
-              false, iip);
+          HdfsFileStatus stat =
+              FSDirStatAndListingOp.createFileStatusForEditLog(fsDir, iip);
           fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
               addCloseOp.rpcCallId, stat);
         }
@@ -402,10 +400,8 @@ public class FSEditLogLoader {
               false);
           // add the op into retry cache if necessary
           if (toAddRetryCache) {
-            HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
-                fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME,
-                HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-                Snapshot.CURRENT_STATE_ID, false, iip);
+            HdfsFileStatus stat =
+                FSDirStatAndListingOp.createFileStatusForEditLog(fsDir, iip);
             fsNamesys.addCacheEntryWithPayload(addCloseOp.rpcClientId,
                 addCloseOp.rpcCallId, new LastBlockWithStatus(lb, stat));
           }
@@ -480,10 +476,8 @@ public class FSEditLogLoader {
             false, false);
         // add the op into retry cache if necessary
         if (toAddRetryCache) {
-          HdfsFileStatus stat = FSDirStatAndListingOp.createFileStatusForEditLog(
-              fsNamesys.dir, path, HdfsFileStatus.EMPTY_NAME,
-              HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED,
-              Snapshot.CURRENT_STATE_ID, false, iip);
+          HdfsFileStatus stat =
+              FSDirStatAndListingOp.createFileStatusForEditLog(fsDir, iip);
           fsNamesys.addCacheEntryWithPayload(appendOp.rpcClientId,
               appendOp.rpcCallId, new LastBlockWithStatus(lb, stat));
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
index 2e0775b..2f9bc37 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeAttributeProvider.java
@@ -17,13 +17,6 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.Set;
-
-import com.google.common.annotations.VisibleForTesting;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
@@ -87,7 +80,7 @@ public abstract class INodeAttributeProvider {
    */
   public abstract void stop();
 
-  @VisibleForTesting
+  @Deprecated
   String[] getPathElements(String path) {
     path = path.trim();
     if (path.charAt(0) != Path.SEPARATOR_CHAR) {
@@ -115,6 +108,7 @@ public abstract class INodeAttributeProvider {
     return pathElements;
   }
 
+  @Deprecated
   public INodeAttributes getAttributes(String fullPath, INodeAttributes inode) {
     return getAttributes(getPathElements(fullPath), inode);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
index 8f65ff8..04d3bda 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodesInPath.java
@@ -399,7 +399,7 @@ public class INodesInPath {
    */
   private INodesInPath getAncestorINodesInPath(int length) {
     Preconditions.checkArgument(length >= 0 && length < inodes.length);
-    Preconditions.checkState(!isSnapshot());
+    Preconditions.checkState(isDotSnapshotDir() || !isSnapshot());
     final INode[] anodes = new INode[length];
     final byte[][] apath = new byte[length][];
     System.arraycopy(this.inodes, 0, anodes, 0, length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/a0730aa5/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
index 24ec1a2..3a318bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestSnapshotPathINodes.java
@@ -166,6 +166,9 @@ public class TestSnapshotPathINodes {
     assertEquals(sub1.toString(), nodesInPath.getPath(2));
     assertEquals(file1.toString(), nodesInPath.getPath(3));
 
+    assertEquals(file1.getParent().toString(),
+                 nodesInPath.getParentINodesInPath().getPath());
+
     nodesInPath = INodesInPath.resolve(fsdir.rootDir, components, false);
     assertEquals(nodesInPath.length(), components.length);
     assertSnapshot(nodesInPath, false, null, -1);
@@ -212,6 +215,9 @@ public class TestSnapshotPathINodes {
     // The number of INodes returned should still be components.length
     // since we put a null in the inode array for ".snapshot"
     assertEquals(nodesInPath.length(), components.length);
+    // ensure parent inodes can strip the .snapshot
+    assertEquals(sub1.toString(),
+        nodesInPath.getParentINodesInPath().getPath());
 
     // No SnapshotRoot dir is included in the resolved inodes  
     assertSnapshot(nodesInPath, true, snapshot, -1);


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