Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 1A8A5200B72 for ; Fri, 26 Aug 2016 22:35:11 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 18E6E160AC4; Fri, 26 Aug 2016 20:35:11 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id C3616160A94 for ; Fri, 26 Aug 2016 22:35:09 +0200 (CEST) Received: (qmail 16916 invoked by uid 500); 26 Aug 2016 20:35:06 -0000 Mailing-List: contact common-commits-help@hadoop.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list common-commits@hadoop.apache.org Received: (qmail 16579 invoked by uid 99); 26 Aug 2016 20:35:06 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 26 Aug 2016 20:35:06 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 06B43EEE38; Fri, 26 Aug 2016 20:35:06 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: aw@apache.org To: common-commits@hadoop.apache.org Date: Fri, 26 Aug 2016 20:35:13 -0000 Message-Id: <67ab645198cc42e88be4f2536b0b8356@git.apache.org> In-Reply-To: References: X-Mailer: ASF-Git Admin Mailer Subject: [09/52] [abbrv] hadoop git commit: HDFS-10762. Pass IIP for file status related methods. Contributed by Daryn Sharp. archived-at: Fri, 26 Aug 2016 20:35:11 -0000 HDFS-10762. Pass IIP for file status related methods. 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/22fc46d7 Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/22fc46d7 Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/22fc46d7 Branch: refs/heads/HADOOP-13341 Commit: 22fc46d7659972ff016ccf1c6f781f0c160be26f Parents: dc7a1c5 Author: Kihwal Lee Authored: Mon Aug 22 15:37:02 2016 -0500 Committer: Kihwal Lee Committed: Mon Aug 22 15:37:02 2016 -0500 ---------------------------------------------------------------------- .../hdfs/server/namenode/FSDirAppendOp.java | 6 +- .../server/namenode/FSDirStatAndListingOp.java | 80 +++++++++----------- .../hdfs/server/namenode/FSDirWriteFileOp.java | 3 +- .../hdfs/server/namenode/FSDirectory.java | 14 ++-- .../hdfs/server/namenode/INodesInPath.java | 42 ++++++++-- .../hadoop/hdfs/TestReservedRawPaths.java | 21 +++++ 6 files changed, 102 insertions(+), 64 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/hadoop/blob/22fc46d7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java index 3a5d7dc..5192352 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java @@ -85,9 +85,10 @@ final class FSDirAppendOp { final LocatedBlock lb; final FSDirectory fsd = fsn.getFSDirectory(); final String src; + final INodesInPath iip; fsd.writeLock(); try { - final INodesInPath iip = fsd.resolvePathForWrite(pc, srcArg); + iip = fsd.resolvePathForWrite(pc, srcArg); src = iip.getPath(); // Verify that the destination does not exist as a directory already final INode inode = iip.getLastINode(); @@ -148,8 +149,7 @@ final class FSDirAppendOp { fsd.writeUnlock(); } - HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, src, false, - FSDirectory.isReservedRawName(srcArg)); + HdfsFileStatus stat = FSDirStatAndListingOp.getFileInfo(fsd, iip); if (lb != null) { NameNode.stateChangeLog.debug( "DIR* NameSystem.appendFile: file {} for {} at {} block {} block" http://git-wip-us.apache.org/repos/asf/hadoop/blob/22fc46d7/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 c9eedf5..88be510 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 @@ -108,16 +108,16 @@ class FSDirStatAndListingOp { if (!DFSUtil.isValidName(src)) { throw new InvalidPathException("Invalid file name: " + src); } + final INodesInPath iip; if (fsd.isPermissionEnabled()) { FSPermissionChecker pc = fsd.getPermissionChecker(); - final INodesInPath iip = fsd.resolvePath(pc, srcArg, resolveLink); - src = iip.getPath(); + iip = fsd.resolvePath(pc, srcArg, resolveLink); fsd.checkPermission(pc, iip, false, null, null, null, null, false); } else { src = FSDirectory.resolvePath(srcArg, fsd); + iip = fsd.getINodesInPath(src, resolveLink); } - return getFileInfo(fsd, src, FSDirectory.isReservedRawName(srcArg), - resolveLink); + return getFileInfo(fsd, iip); } /** @@ -230,7 +230,6 @@ class FSDirStatAndListingOp { String src, byte[] startAfter, boolean needLocation, boolean isSuperUser) throws IOException { String srcs = FSDirectory.normalizePath(src); - final boolean isRawPath = FSDirectory.isReservedRawName(src); if (FSDirectory.isExactReservedName(srcs)) { return getReservedListing(fsd); } @@ -257,7 +256,7 @@ class FSDirStatAndListingOp { return new DirectoryListing( new HdfsFileStatus[]{ createFileStatus( fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs, - needLocation, parentStoragePolicy, snapshot, isRawPath, iip) + needLocation, parentStoragePolicy, iip) }, 0); } @@ -282,7 +281,7 @@ class FSDirStatAndListingOp { cur.getLocalNameBytes()); listing[i] = createFileStatus(fsd, cur.getLocalNameBytes(), nodeAttrs, needLocation, getStoragePolicyID(curPolicy, parentStoragePolicy), - snapshot, isRawPath, iipWithChild); + iipWithChild); listingCnt++; if (needLocation) { // Once we hit lsLimit locations, stop. @@ -339,7 +338,6 @@ class FSDirStatAndListingOp { listing[i] = createFileStatus( fsd, sRoot.getLocalNameBytes(), nodeAttrs, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, - Snapshot.CURRENT_STATE_ID, false, INodesInPath.fromINode(sRoot)); } return new DirectoryListing( @@ -363,10 +361,8 @@ class FSDirStatAndListingOp { * @return object containing information regarding the file * or null if file not found */ - static HdfsFileStatus getFileInfo( - FSDirectory fsd, String path, INodesInPath iip, boolean isRawPath, - boolean includeStoragePolicy) - throws IOException { + static HdfsFileStatus getFileInfo(FSDirectory fsd, + INodesInPath iip, boolean includeStoragePolicy) throws IOException { fsd.readLock(); try { final INode node = iip.getLastINode(); @@ -377,36 +373,32 @@ class FSDirStatAndListingOp { byte policyId = includeStoragePolicy && !node.isSymlink() ? node.getStoragePolicyID() : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED; - INodeAttributes nodeAttrs = getINodeAttributes(fsd, path, + INodeAttributes nodeAttrs = getINodeAttributes(fsd, iip.getPath(), HdfsFileStatus.EMPTY_NAME, node, iip.getPathSnapshotId()); return createFileStatus(fsd, HdfsFileStatus.EMPTY_NAME, nodeAttrs, - policyId, iip.getPathSnapshotId(), isRawPath, iip); + policyId, iip); } finally { fsd.readUnlock(); } } - static HdfsFileStatus getFileInfo( - FSDirectory fsd, String src, boolean resolveLink, boolean isRawPath) + static HdfsFileStatus getFileInfo(FSDirectory fsd, INodesInPath iip) throws IOException { - fsd.readLock(); - try { - HdfsFileStatus status = null; - final INodesInPath iip = fsd.getINodesInPath(src, resolveLink); - if (FSDirectory.isExactReservedName(iip.getPathComponents())) { - status = FSDirectory.DOT_RESERVED_STATUS; - } else if (iip.isDotSnapshotDir()) { - if (fsd.getINode4DotSnapshot(iip) != null) { - status = FSDirectory.DOT_SNAPSHOT_DIR_STATUS; - } - } else { - status = getFileInfo(fsd, src, iip, isRawPath, true); + if (FSDirectory.isExactReservedName(iip.getPathComponents())) { + return FSDirectory.DOT_RESERVED_STATUS; + } + + if (iip.isDotSnapshotDir()) { + if (fsd.getINode4DotSnapshot(iip) != null) { + return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null, + HdfsFileStatus.EMPTY_NAME, -1L, 0, null, + HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); } - return status; - } finally { - fsd.readUnlock(); + return null; } + + return getFileInfo(fsd, iip, true); } /** @@ -423,15 +415,12 @@ class FSDirStatAndListingOp { */ private static HdfsFileStatus createFileStatus( FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs, - boolean needLocation, byte storagePolicy, int snapshot, boolean isRawPath, - INodesInPath iip) + boolean needLocation, byte storagePolicy, INodesInPath iip) throws IOException { if (needLocation) { - return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy, - snapshot, isRawPath, iip); + return createLocatedFileStatus(fsd, path, nodeAttrs, storagePolicy, iip); } else { - return createFileStatus(fsd, path, nodeAttrs, storagePolicy, - snapshot, isRawPath, iip); + return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip); } } @@ -445,8 +434,7 @@ class FSDirStatAndListingOp { INodesInPath iip) throws IOException { INodeAttributes nodeAttrs = getINodeAttributes( fsd, fullPath, path, iip.getLastINode(), snapshot); - return createFileStatus(fsd, path, nodeAttrs, storagePolicy, - snapshot, isRawPath, iip); + return createFileStatus(fsd, path, nodeAttrs, storagePolicy, iip); } /** @@ -454,14 +442,15 @@ class FSDirStatAndListingOp { * @param iip the INodesInPath containing the target INode and its ancestors */ static HdfsFileStatus createFileStatus( - FSDirectory fsd, byte[] path, - INodeAttributes nodeAttrs, byte storagePolicy, int snapshot, - boolean isRawPath, INodesInPath iip) throws IOException { + 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; 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); @@ -511,10 +500,9 @@ class FSDirStatAndListingOp { * Create FileStatus with location info by file INode * @param iip the INodesInPath containing the target INode and its ancestors */ - private static HdfsLocatedFileStatus createLocatedFileStatus( + private static HdfsFileStatus createLocatedFileStatus( FSDirectory fsd, byte[] path, INodeAttributes nodeAttrs, - byte storagePolicy, int snapshot, - boolean isRawPath, INodesInPath iip) throws IOException { + byte storagePolicy, INodesInPath iip) throws IOException { assert fsd.hasReadLock(); long size = 0; // length is zero for directories short replication = 0; @@ -522,6 +510,8 @@ class FSDirStatAndListingOp { 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); http://git-wip-us.apache.org/repos/asf/hadoop/blob/22fc46d7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 8429024..030d8cc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -340,7 +340,6 @@ class FSDirWriteFileOp { version = ezInfo.protocolVersion; } - boolean isRawPath = FSDirectory.isReservedRawName(src); FSDirectory fsd = fsn.getFSDirectory(); INodesInPath iip = fsd.resolvePathForWrite(pc, src); src = iip.getPath(); @@ -444,7 +443,7 @@ class FSDirWriteFileOp { NameNode.stateChangeLog.debug("DIR* NameSystem.startFile: added " + src + " inode " + newNode.getId() + " " + holder); } - return FSDirStatAndListingOp.getFileInfo(fsd, src, false, isRawPath); + return FSDirStatAndListingOp.getFileInfo(fsd, iip); } static EncryptionKeyInfo getEncryptionKeyInfo(FSNamesystem fsn, http://git-wip-us.apache.org/repos/asf/hadoop/blob/22fc46d7/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 b6652e4..5dc5a03 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 @@ -531,21 +531,24 @@ public class FSDirectory implements Closeable { * @throws FileNotFoundException * @throws AccessControlException */ - INodesInPath resolvePath(FSPermissionChecker pc, String src) + @VisibleForTesting + public INodesInPath resolvePath(FSPermissionChecker pc, String src) throws UnresolvedLinkException, FileNotFoundException, AccessControlException { return resolvePath(pc, src, true); } - INodesInPath resolvePath(FSPermissionChecker pc, String src, + @VisibleForTesting + public INodesInPath resolvePath(FSPermissionChecker pc, String src, boolean resolveLink) throws UnresolvedLinkException, FileNotFoundException, AccessControlException { byte[][] components = INode.getPathComponents(src); - if (isPermissionEnabled && pc != null && isReservedRawName(components)) { + boolean isRaw = isReservedRawName(components); + if (isPermissionEnabled && pc != null && isRaw) { pc.checkSuperuserPrivilege(); } components = resolveComponents(components, this); - return INodesInPath.resolve(rootDir, components, resolveLink); + return INodesInPath.resolve(rootDir, components, isRaw, resolveLink); } INodesInPath resolvePathForWrite(FSPermissionChecker pc, String src) @@ -1662,8 +1665,7 @@ public class FSDirectory implements Closeable { HdfsFileStatus getAuditFileInfo(INodesInPath iip) throws IOException { return (namesystem.isAuditEnabled() && namesystem.isExternalInvocation()) - ? FSDirStatAndListingOp.getFileInfo(this, iip.getPath(), iip, false, - false) : null; + ? FSDirStatAndListingOp.getFileInfo(this, iip, false) : null; } /** http://git-wip-us.apache.org/repos/asf/hadoop/blob/22fc46d7/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 86cab28..af8998f 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 @@ -126,6 +126,12 @@ public class INodesInPath { static INodesInPath resolve(final INodeDirectory startingDir, final byte[][] components, final boolean resolveLink) throws UnresolvedLinkException { + return resolve(startingDir, components, false, resolveLink); + } + + static INodesInPath resolve(final INodeDirectory startingDir, + final byte[][] components, final boolean isRaw, + final boolean resolveLink) throws UnresolvedLinkException { Preconditions.checkArgument(startingDir.compareTo(components[0]) == 0); INode curNode = startingDir; @@ -225,7 +231,7 @@ public class INodesInPath { System.arraycopy(inodes, 0, newNodes, 0, newNodes.length); inodes = newNodes; } - return new INodesInPath(inodes, components, isSnapshot, snapshotId); + return new INodesInPath(inodes, components, isRaw, isSnapshot, snapshotId); } private static boolean shouldUpdateLatestId(int sid, int snapshotId) { @@ -249,7 +255,8 @@ public class INodesInPath { INode[] inodes = new INode[iip.inodes.length]; System.arraycopy(iip.inodes, 0, inodes, 0, inodes.length); inodes[pos] = inode; - return new INodesInPath(inodes, iip.path, iip.isSnapshot, iip.snapshotId); + return new INodesInPath(inodes, iip.path, iip.isRaw, + iip.isSnapshot, iip.snapshotId); } /** @@ -267,7 +274,8 @@ public class INodesInPath { byte[][] path = new byte[iip.path.length + 1][]; System.arraycopy(iip.path, 0, path, 0, path.length - 1); path[path.length - 1] = childName; - return new INodesInPath(inodes, path, iip.isSnapshot, iip.snapshotId); + return new INodesInPath(inodes, path, iip.isRaw, + iip.isSnapshot, iip.snapshotId); } private final byte[][] path; @@ -279,6 +287,13 @@ public class INodesInPath { * true if this path corresponds to a snapshot */ private final boolean isSnapshot; + + /** + * true if this is a /.reserved/raw path. path component resolution strips + * it from the path so need to track it separately. + */ + private final boolean isRaw; + /** * For snapshot paths, it is the id of the snapshot; or * {@link Snapshot#CURRENT_STATE_ID} if the snapshot does not exist. For @@ -287,17 +302,18 @@ public class INodesInPath { */ private final int snapshotId; - private INodesInPath(INode[] inodes, byte[][] path, boolean isSnapshot, - int snapshotId) { + private INodesInPath(INode[] inodes, byte[][] path, boolean isRaw, + boolean isSnapshot,int snapshotId) { Preconditions.checkArgument(inodes != null && path != null); this.inodes = inodes; this.path = path; + this.isRaw = isRaw; this.isSnapshot = isSnapshot; this.snapshotId = snapshotId; } private INodesInPath(INode[] inodes, byte[][] path) { - this(inodes, path, false, CURRENT_STATE_ID); + this(inodes, path, false, false, CURRENT_STATE_ID); } /** @@ -400,7 +416,7 @@ public class INodesInPath { final byte[][] apath = new byte[length][]; System.arraycopy(this.inodes, 0, anodes, 0, length); System.arraycopy(this.path, 0, apath, 0, length); - return new INodesInPath(anodes, apath, false, snapshotId); + return new INodesInPath(anodes, apath, isRaw, false, snapshotId); } /** @@ -428,7 +444,7 @@ public class INodesInPath { byte[][] existingPath = new byte[i][]; System.arraycopy(inodes, 0, existing, 0, i); System.arraycopy(path, 0, existingPath, 0, i); - return new INodesInPath(existing, existingPath, false, snapshotId); + return new INodesInPath(existing, existingPath, isRaw, false, snapshotId); } /** @@ -438,10 +454,20 @@ public class INodesInPath { return this.isSnapshot; } + /** + * @return if .snapshot is the last path component. + */ boolean isDotSnapshotDir() { return isDotSnapshotDir(getLastLocalName()); } + /** + * @return if this is a /.reserved/raw path. + */ + public boolean isRaw() { + return isRaw; + } + private static String toString(INode inode) { return inode == null? null: inode.getLocalName(); } http://git-wip-us.apache.org/repos/asf/hadoop/blob/22fc46d7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java ---------------------------------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java index 449f715..c09d346 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReservedRawPaths.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hdfs.client.CreateEncryptionZoneFlag; import org.apache.hadoop.hdfs.client.HdfsAdmin; import org.apache.hadoop.hdfs.server.namenode.EncryptionZoneManager; import org.apache.hadoop.hdfs.server.namenode.FSDirectory; +import org.apache.hadoop.hdfs.server.namenode.INodesInPath; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.log4j.Level; @@ -49,6 +50,8 @@ import static org.apache.hadoop.hdfs.DFSTestUtil.verifyFilesNotEqual; import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; import static org.apache.hadoop.test.GenericTestUtils.assertMatches; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; public class TestReservedRawPaths { @@ -100,6 +103,24 @@ public class TestReservedRawPaths { } /** + * Verify resolving path will return an iip that tracks if the original + * path was a raw path. + */ + @Test(timeout = 120000) + public void testINodesInPath() throws IOException { + FSDirectory fsd = cluster.getNamesystem().getFSDirectory(); + final String path = "/path"; + + INodesInPath iip = fsd.resolvePath(null, path); + assertFalse(iip.isRaw()); + assertEquals(path, iip.getPath()); + + iip = fsd.resolvePath(null, "/.reserved/raw" + path); + assertTrue(iip.isRaw()); + assertEquals(path, iip.getPath()); + } + + /** * Basic read/write tests of raw files. * Create a non-encrypted file * Create an encryption zone --------------------------------------------------------------------- To unsubscribe, e-mail: common-commits-unsubscribe@hadoop.apache.org For additional commands, e-mail: common-commits-help@hadoop.apache.org