From e9c0bd2a0fb3c06be27006df587f873628f0c9ad Mon Sep 17 00:00:00 2001 From: Tom McCormick Date: Mon, 21 Nov 2022 09:58:25 -0800 Subject: [PATCH] Add additional contract tests for getEnclosingRoot --- .../apache/hadoop/fs/AbstractFileSystem.java | 184 ++++++++++-------- .../org/apache/hadoop/fs/viewfs/ViewFs.java | 14 ++ .../hadoop/fs/TestGetEnclosingRoot.java | 95 +++++++++ .../AbstractContractGetEnclosingRoot.java | 100 ++++++++++ .../TestLocalFSContractGetEnclosingRoot.java | 32 +++ .../TestRawlocalContractGetEnclosingRoot.java | 32 +++ .../fs/viewfs/TestViewFileSystemHdfs.java | 31 +++ 7 files changed, 405 insertions(+), 83 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java index a4737c548c8fa9..59bdbee3aae573 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/AbstractFileSystem.java @@ -65,9 +65,9 @@ * This class provides an interface for implementors of a Hadoop file system * (analogous to the VFS of Unix). Applications do not access this class; * instead they access files across all file systems using {@link FileContext}. - * + * * Pathnames passed to AbstractFileSystem can be fully qualified URI that - * matches the "this" file system (ie same scheme and authority) + * matches the "this" file system (ie same scheme and authority) * or a Slash-relative name that is assumed to be relative * to the root of the "this" file system . */ @@ -77,34 +77,34 @@ public abstract class AbstractFileSystem implements PathCapabilities { static final Logger LOG = LoggerFactory.getLogger(AbstractFileSystem.class); /** Recording statistics per a file system class. */ - private static final Map + private static final Map STATISTICS_TABLE = new HashMap(); - + /** Cache of constructors for each file system class. */ - private static final Map, Constructor> CONSTRUCTOR_CACHE = + private static final Map, Constructor> CONSTRUCTOR_CACHE = new ConcurrentHashMap, Constructor>(); - - private static final Class[] URI_CONFIG_ARGS = + + private static final Class[] URI_CONFIG_ARGS = new Class[]{URI.class, Configuration.class}; - + /** The statistics for this file system. */ protected Statistics statistics; @VisibleForTesting static final String NO_ABSTRACT_FS_ERROR = "No AbstractFileSystem configured for scheme"; - + private final URI myUri; - + public Statistics getStatistics() { return statistics; } - + /** * Returns true if the specified string is considered valid in the path part * of a URI by this file system. The default implementation enforces the rules * of HDFS, but subclasses may override this method to implement specific * validation rules for specific file systems. - * + * * @param src String source filename to check, path part of the URI * @return boolean true if the specified string is considered valid */ @@ -121,8 +121,8 @@ public boolean isValidName(String src) { } return true; } - - /** + + /** * Create an object for the given class and initialize it from conf. * @param theClass class of which an object is created * @param conf Configuration @@ -152,7 +152,7 @@ static T newInstance(Class theClass, } return result; } - + /** * Create a file system instance for the specified uri using the conf. The * conf is used to find the class name that implements the file system. The @@ -160,7 +160,7 @@ static T newInstance(Class theClass, * * @param uri URI of the file system * @param conf Configuration for the file system - * + * * @return Returns the file system for the given URI * * @throws UnsupportedFileSystemException file system for uri is @@ -182,7 +182,7 @@ public static AbstractFileSystem createFileSystem(URI uri, Configuration conf) /** * Get the statistics for a particular file system. - * + * * @param uri * used as key to lookup STATISTICS_TABLE. Only scheme and authority * part of the uri are used. @@ -202,7 +202,7 @@ protected static synchronized Statistics getStatistics(URI uri) { } return result; } - + private static URI getBaseUri(URI uri) { String scheme = uri.getScheme(); String authority = uri.getAuthority(); @@ -214,7 +214,7 @@ private static URI getBaseUri(URI uri) { } return URI.create(baseUriString); } - + public static synchronized void clearStatistics() { for(Statistics stat: STATISTICS_TABLE.values()) { stat.reset(); @@ -230,7 +230,7 @@ public static synchronized void printStatistics() { + pair.getKey().getAuthority() + ": " + pair.getValue()); } } - + protected static synchronized Map getAllStatistics() { Map statsMap = new HashMap( STATISTICS_TABLE.size()); @@ -249,14 +249,14 @@ protected static synchronized Map getAllStatistics() { * determines a configuration property name, * fs.AbstractFileSystem.scheme.impl whose value names the * AbstractFileSystem class. - * + * * The entire URI and conf is passed to the AbstractFileSystem factory method. - * + * * @param uri for the file system to be created. * @param conf which is passed to the file system impl. - * + * * @return file system for the given URI. - * + * * @throws UnsupportedFileSystemException if the file system for * uri is not supported. */ @@ -267,7 +267,7 @@ public static AbstractFileSystem get(final URI uri, final Configuration conf) /** * Constructor to be called by subclasses. - * + * * @param uri for this file system. * @param supportedScheme the scheme supported by the implementor * @param authorityNeeded if true then theURI must have authority, if false @@ -279,7 +279,7 @@ public AbstractFileSystem(final URI uri, final String supportedScheme, final boolean authorityNeeded, final int defaultPort) throws URISyntaxException { myUri = getUri(uri, supportedScheme, authorityNeeded, defaultPort); - statistics = getStatistics(uri); + statistics = getStatistics(uri); } /** @@ -303,14 +303,14 @@ public void checkScheme(URI uri, String supportedScheme) { * Get the URI for the file system based on the given URI. The path, query * part of the given URI is stripped out and default file system port is used * to form the URI. - * + * * @param uri FileSystem URI. * @param authorityNeeded if true authority cannot be null in the URI. If * false authority must be null. * @param defaultPort default port to use if port is not specified in the URI. - * + * * @return URI of the file system - * + * * @throws URISyntaxException uri has syntax error */ private URI getUri(URI uri, String supportedScheme, @@ -329,7 +329,7 @@ private URI getUri(URI uri, String supportedScheme, throw new HadoopIllegalArgumentException("Uri without authority: " + uri); } else { return new URI(supportedScheme + ":///"); - } + } } // authority is non null - AuthorityNeeded may be true or false. int port = uri.getPort(); @@ -339,10 +339,10 @@ private URI getUri(URI uri, String supportedScheme, } return new URI(supportedScheme + "://" + uri.getHost() + ":" + port); } - + /** * The default port of this file system. - * + * * @return default port of this file system's Uri scheme * A uri with a port of -1 => default port; */ @@ -350,18 +350,18 @@ private URI getUri(URI uri, String supportedScheme, /** * Returns a URI whose scheme and authority identify this FileSystem. - * + * * @return the uri of this file system. */ public URI getUri() { return myUri; } - + /** * Check that a Path belongs to this FileSystem. - * + * * If the path is fully qualified URI, then its scheme and authority - * matches that of this file system. Otherwise the path must be + * matches that of this file system. Otherwise the path must be * slash-relative name. * @param path the path. * @throws InvalidPathException if the path is invalid @@ -375,7 +375,7 @@ public void checkPath(Path path) { if (path.isUriPathAbsolute()) { return; } - throw new InvalidPathException("relative paths not allowed:" + + throw new InvalidPathException("relative paths not allowed:" + path); } else { throw new InvalidPathException( @@ -385,17 +385,17 @@ public void checkPath(Path path) { String thisScheme = this.getUri().getScheme(); String thisHost = this.getUri().getHost(); String thatHost = uri.getHost(); - + // Schemes and hosts must match. // Allow for null Authority for file:/// if (!thisScheme.equalsIgnoreCase(thatScheme) || - (thisHost != null && + (thisHost != null && !thisHost.equalsIgnoreCase(thatHost)) || (thisHost == null && thatHost != null)) { throw new InvalidPathException("Wrong FS: " + path + ", expected: " + this.getUri()); } - + // Ports must match, unless this FS instance is using the default port, in // which case the port may be omitted from the given URI int thisPort = this.getUri().getPort(); @@ -411,13 +411,13 @@ public void checkPath(Path path) { + " with port=" + thisPort); } } - + /** * Get the path-part of a pathname. Checks that URI matches this file system * and that the path-part is a valid name. - * + * * @param p path - * + * * @return path-part of the Path p */ public String getUriPath(final Path p) { @@ -429,7 +429,7 @@ public String getUriPath(final Path p) { } return s; } - + /** * Make the path fully qualified to this file system * @param path the path. @@ -439,23 +439,23 @@ public Path makeQualified(Path path) { checkPath(path); return path.makeQualified(this.getUri(), null); } - + /** * Some file systems like LocalFileSystem have an initial workingDir * that is used as the starting workingDir. For other file systems * like HDFS there is no built in notion of an initial workingDir. - * + * * @return the initial workingDir if the file system has such a notion * otherwise return a null. */ public Path getInitialWorkingDirectory() { return null; } - - /** + + /** * Return the current user's home directory in this file system. * The default implementation returns "/user/$USER/". - * + * * @return current user's home directory. */ public Path getHomeDirectory() { @@ -470,17 +470,17 @@ public Path getHomeDirectory() { return new Path("/user/" + username) .makeQualified(getUri(), null); } - + /** * Return a set of server default configuration values. - * + * * @return server default configuration values - * + * * @throws IOException an I/O error occurred * @deprecated use {@link #getServerDefaults(Path)} instead */ @Deprecated - public abstract FsServerDefaults getServerDefaults() throws IOException; + public abstract FsServerDefaults getServerDefaults() throws IOException; /** * Return a set of server default configuration values based on path. @@ -496,7 +496,7 @@ public FsServerDefaults getServerDefaults(final Path f) throws IOException { * Return the fully-qualified path of path f resolving the path * through any internal symlinks or mount point * @param p path to be resolved - * @return fully qualified path + * @return fully qualified path * @throws FileNotFoundException when file not find throw. * @throws AccessControlException when accees control error throw. * @throws IOException raised on errors performing I/O. @@ -508,7 +508,7 @@ public Path resolvePath(final Path p) throws FileNotFoundException, checkPath(p); return getFileStatus(p).getPath(); // default impl is to return the path } - + /** * The specification of this method matches that of * {@link FileContext#create(Path, EnumSet, Options.CreateOpts...)} except @@ -541,7 +541,7 @@ public final FSDataOutputStream create(final Path f, FsPermission permission = null; Progressable progress = null; Boolean createParent = null; - + for (CreateOpts iOpt : opts) { if (CreateOpts.BlockSize.class.isInstance(iOpt)) { if (blockSize != -1) { @@ -603,10 +603,10 @@ public final FSDataOutputStream create(final Path f, FsServerDefaults ssDef = getServerDefaults(f); if (ssDef.getBlockSize() % ssDef.getBytesPerChecksum() != 0) { - throw new IOException("Internal error: default blockSize is" + + throw new IOException("Internal error: default blockSize is" + " not a multiple of default bytesPerChecksum "); } - + if (blockSize == -1) { blockSize = ssDef.getBlockSize(); } @@ -673,7 +673,7 @@ public abstract FSDataOutputStream createInternal(Path f, /** * The specification of this method matches that of * {@link FileContext#mkdir(Path, FsPermission, boolean)} except that the Path - * f must be fully qualified and the permission is absolute (i.e. + * f must be fully qualified and the permission is absolute (i.e. * umask has been applied). * @param dir directory. * @param permission permission. @@ -806,12 +806,12 @@ public final void rename(final Path src, final Path dst, } renameInternal(src, dst, overwrite); } - + /** * The specification of this method matches that of * {@link FileContext#rename(Path, Path, Options.Rename...)} except that Path * f must be for this file system and NO OVERWRITE is performed. - * + * * File systems that do not have a built in overwrite need implement only this * method and can take advantage of the default impl of the other * {@link #renameInternal(Path, Path, boolean)} @@ -829,7 +829,7 @@ public abstract void renameInternal(final Path src, final Path dst) throws AccessControlException, FileAlreadyExistsException, FileNotFoundException, ParentNotDirectoryException, UnresolvedLinkException, IOException; - + /** * The specification of this method matches that of * {@link FileContext#rename(Path, Path, Options.Rename...)} except that Path @@ -895,7 +895,7 @@ public void renameInternal(final Path src, final Path dst, } renameInternal(src, dst); } - + /** * Returns true if the file system supports symlinks, false otherwise. * @return true if filesystem supports symlinks @@ -903,9 +903,9 @@ public void renameInternal(final Path src, final Path dst, public boolean supportsSymlinks() { return false; } - + /** - * The specification of this method matches that of + * The specification of this method matches that of * {@link FileContext#createSymlink(Path, Path, boolean)}; * * @param target target. @@ -916,7 +916,7 @@ public boolean supportsSymlinks() { */ public void createSymlink(final Path target, final Path link, final boolean createParent) throws IOException, UnresolvedLinkException { - throw new IOException("File system does not support symlinks"); + throw new IOException("File system does not support symlinks"); } /** @@ -925,14 +925,14 @@ public void createSymlink(final Path target, final Path link, * {@link FileContext#getLinkTarget(Path)}. * @param f the path. * @return target path. - * @throws IOException subclass implementations may throw IOException + * @throws IOException subclass implementations may throw IOException */ public Path getLinkTarget(final Path f) throws IOException { throw new AssertionError("Implementation Error: " + getClass() + " that threw an UnresolvedLinkException, causing this method to be" + " called, needs to override this method."); } - + /** * The specification of this method matches that of * {@link FileContext#setPermission(Path, FsPermission)} except that Path f @@ -998,11 +998,11 @@ public abstract void setTimes(final Path f, final long mtime, public abstract FileChecksum getFileChecksum(final Path f) throws AccessControlException, FileNotFoundException, UnresolvedLinkException, IOException; - + /** * The specification of this method matches that of - * {@link FileContext#getFileStatus(Path)} - * except that an UnresolvedLinkException may be thrown if a symlink is + * {@link FileContext#getFileStatus(Path)} + * except that an UnresolvedLinkException may be thrown if a symlink is * encountered in the path. * * @param f the path. @@ -1052,7 +1052,7 @@ public void access(Path path, FsAction mode) throws AccessControlException, /** * The specification of this method matches that of * {@link FileContext#getFileLinkStatus(Path)} - * except that an UnresolvedLinkException may be thrown if a symlink is + * except that an UnresolvedLinkException may be thrown if a symlink is * encountered in the path leading up to the final path component. * If the file system does not support symlinks then the behavior is * equivalent to {@link AbstractFileSystem#getFileStatus(Path)}. @@ -1105,7 +1105,7 @@ public FsStatus getFsStatus(final Path f) throws AccessControlException, // default impl gets FsStatus of root return getFsStatus(); } - + /** * The specification of this method matches that of * {@link FileContext#getFsStatus(Path)}. @@ -1136,12 +1136,12 @@ public RemoteIterator listStatusIterator(final Path f) return new RemoteIterator() { private int i = 0; private FileStatus[] statusList = listStatus(f); - + @Override public boolean hasNext() { return i < statusList.length; } - + @Override public FileStatus next() { if (!hasNext()) { @@ -1154,7 +1154,7 @@ public FileStatus next() { /** * The specification of this method matches that of - * {@link FileContext#listLocatedStatus(Path)} except that Path f + * {@link FileContext#listLocatedStatus(Path)} except that Path f * must be for this file system. * * In HDFS implementation, the BlockLocation of returned LocatedFileStatus @@ -1174,12 +1174,12 @@ public RemoteIterator listLocatedStatus(final Path f) UnresolvedLinkException, IOException { return new RemoteIterator() { private RemoteIterator itor = listStatusIterator(f); - + @Override public boolean hasNext() throws IOException { return itor.hasNext(); } - + @Override public LocatedFileStatus next() throws IOException { if (!hasNext()) { @@ -1198,7 +1198,7 @@ public LocatedFileStatus next() throws IOException { /** * The specification of this method matches that of - * {@link FileContext.Util#listStatus(Path)} except that Path f must be + * {@link FileContext.Util#listStatus(Path)} except that Path f must be * for this file system. * @param f the path. * @throws AccessControlException access control exception. @@ -1235,7 +1235,7 @@ public RemoteIterator listCorruptFileBlocks(Path path) */ public abstract void setVerifyChecksum(final boolean verifyChecksum) throws AccessControlException, IOException; - + /** * Get a canonical name for this file system. * @return a URI string that uniquely identifies this file system @@ -1243,13 +1243,13 @@ public abstract void setVerifyChecksum(final boolean verifyChecksum) public String getCanonicalServiceName() { return SecurityUtil.buildDTServiceName(getUri(), getUriDefaultPort()); } - + /** * Get one or more delegation tokens associated with the filesystem. Normally * a file system returns a single delegation token. A file system that manages * multiple file systems underneath, could return set of delegation tokens for * all the file systems it manages - * + * * @param renewer the account name that is allowed to renew the token. * @return List of delegation tokens. * If delegation tokens not supported then return a list of size zero. @@ -1577,7 +1577,7 @@ public Collection getAllStoragePolicies() public int hashCode() { return myUri.hashCode(); } - + @Override //Object public boolean equals(Object other) { if (!(other instanceof AbstractFileSystem)) { @@ -1638,6 +1638,24 @@ public MultipartUploaderBuilder createMultipartUploader(Path basePath) return null; } + /** + * Return path of the enclosing root for a given path + * The enclosing root path is a common ancestor that should be used for temp and staging dirs + * as well as within encryption zones and other restricted directories. + * + * Call makeQualified on the param path to ensure the param path to ensure its part of the correct filesystem + * + * @param path file path to find the enclosing root path for + * @return a path to the enclosing root + * @throws IOException + */ + @InterfaceAudience.Public + @InterfaceStability.Unstable + public Path getEnclosingRoot(Path path) throws IOException { + this.makeQualified(path); + return this.makeQualified(new Path("/")); + } + /** * Helper method that throws an {@link UnsupportedOperationException} for the * current {@link FileSystem} method being called. diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java index 5f54c9cdd06aaa..0107431fb0739a 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java @@ -36,6 +36,7 @@ import java.util.Set; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.util.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; @@ -1477,5 +1478,18 @@ public void setStoragePolicy(Path path, String policyName) throws IOException { throw readOnlyMountTable("setStoragePolicy", path); } + + @Override + public Path getEnclosingRoot(Path path) throws IOException { + InodeTree.ResolveResult res; + try { + res = fsState.resolve((path.toString()), true); + } catch (FileNotFoundException ex) { + throw new NotInMountpointException(path, "getEnclosingRoot"); + } + Path fullPath = new Path(res.resolvedPath); + Path enclosingPath = res.targetFileSystem.getEnclosingRoot(path); + return enclosingPath.depth() > fullPath.depth() ? enclosingPath : fullPath; + } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java new file mode 100644 index 00000000000000..8cccbf750ffe8b --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/TestGetEnclosingRoot.java @@ -0,0 +1,95 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Test; + +import static org.junit.Assert.*; + + +public class TestGetEnclosingRoot { + @Test + public void testEnclosingRootEquivalence() throws IOException { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = path("/foo/bar"); + + assertEquals(fs.getEnclosingRoot(foobar), root); + assertEquals(fs.getEnclosingRoot(fs.getEnclosingRoot(foobar)), root); + assertEquals(fs.getEnclosingRoot(foobar), fs.getEnclosingRoot(root)); + + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), root); + assertEquals(fs.getEnclosingRoot(fs.getEnclosingRoot(path(foobar.toString()))), root); + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), fs.getEnclosingRoot(root)); + } + + @Test + public void testEnclosingRootPathExists() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = path("/foo/bar"); + fs.mkdirs(foobar); + + assertEquals(fs.getEnclosingRoot(foobar), root); + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), root); + } + + @Test + public void testEnclosingRootPathDNE() throws Exception { + FileSystem fs = getFileSystem(); + Path foobar = path("/foo/bar"); + Path root = path("/"); + + assertEquals(fs.getEnclosingRoot(foobar), root); + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), root); + } + + @Test + public void testEnclosingRootWrapped() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + + assertEquals(fs.getEnclosingRoot(new Path("/foo/bar")), root); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo"); + Path p = (Path) ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = getFileSystem(); + return wFs.getEnclosingRoot(new Path("/foo/bar")); + }); + assertEquals(p, root); + } + + private FileSystem getFileSystem() throws IOException { + return FileSystem.get(new Configuration()); + } + + /** + * Create a path under the test path provided by + * the FS contract. + * @param filepath path string in + * @return a path qualified by the test filesystem + * @throws IOException IO problems + */ + private Path path(String filepath) throws IOException { + return getFileSystem().makeQualified( + new Path(filepath)); + }} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java new file mode 100644 index 00000000000000..c975d01597f676 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/AbstractContractGetEnclosingRoot.java @@ -0,0 +1,100 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.contract; + +import java.io.IOException; +import java.security.PrivilegedExceptionAction; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + + +public abstract class AbstractContractGetEnclosingRoot extends AbstractFSContractTestBase { + private static final Logger LOG = LoggerFactory.getLogger(AbstractContractGetEnclosingRoot.class); + + @Override + public void setup() throws Exception { + super.setup(); + } + + @Test + public void testEnclosingRootEquivalence() throws IOException { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = path("/foo/bar"); + + assertEquals(fs.getEnclosingRoot(foobar), root); + assertEquals(fs.getEnclosingRoot(fs.getEnclosingRoot(foobar)), root); + assertEquals(fs.getEnclosingRoot(foobar), fs.getEnclosingRoot(root)); + + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), root); + assertEquals(fs.getEnclosingRoot(fs.getEnclosingRoot(path(foobar.toString()))), root); + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), fs.getEnclosingRoot(root)); + } + + @Test + public void testEnclosingRootPathExists() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + Path foobar = path("/foo/bar"); + fs.mkdirs(foobar); + + assertEquals(fs.getEnclosingRoot(foobar), root); + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), root); + } + + @Test + public void testEnclosingRootPathDNE() throws Exception { + FileSystem fs = getFileSystem(); + Path foobar = path("/foo/bar"); + Path root = path("/"); + + assertEquals(fs.getEnclosingRoot(foobar), root); + assertEquals(fs.getEnclosingRoot(path(foobar.toString())), root); + } + + @Test + public void testEnclosingRootWrapped() throws Exception { + FileSystem fs = getFileSystem(); + Path root = path("/"); + + assertEquals(fs.getEnclosingRoot(new Path("/foo/bar")), root); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo"); + Path p = (Path) ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = getContract().getTestFileSystem(); + return wFs.getEnclosingRoot(new Path("/foo/bar")); + }); + assertEquals(p, root); + } + + /** + * Create a path under the test path provided by + * the FS contract. + * @param filepath path string in + * @return a path qualified by the test filesystem + * @throws IOException IO problems + */ + protected Path path(String filepath) throws IOException { + return getFileSystem().makeQualified( + new Path(getContract().getTestPath(), filepath)); + } +} \ No newline at end of file diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java new file mode 100644 index 00000000000000..9819245ba68674 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/localfs/TestLocalFSContractGetEnclosingRoot.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.contract.localfs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractGetEnclosingRoot; +import org.apache.hadoop.fs.contract.AbstractFSContract; + + +public class TestLocalFSContractGetEnclosingRoot + extends AbstractContractGetEnclosingRoot { + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new LocalFSContract(conf); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java new file mode 100644 index 00000000000000..7e99cb7b88ed6a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/contract/rawlocal/TestRawlocalContractGetEnclosingRoot.java @@ -0,0 +1,32 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.fs.contract.rawlocal; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractGetEnclosingRoot; +import org.apache.hadoop.fs.contract.AbstractFSContract; + + +public class TestRawlocalContractGetEnclosingRoot extends AbstractContractGetEnclosingRoot { + + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new RawlocalFSContract(conf); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java index 3f6e7d62db1f73..1ff443df9e9c68 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java @@ -561,4 +561,35 @@ public void testEnclosingRootFailure() throws Exception { LambdaTestUtils.intercept(IllegalArgumentException.class, ()-> fsView.getEnclosingRoot(new Path("hdfs://fakeauthority/"))); } + + @Test + public void testEnclosingRootWrapped() throws Exception { + try { + final Path zone = new Path("/data/EZ"); + fsTarget.mkdirs(zone); + final Path zone1 = new Path("/data/EZ/zone1"); + fsTarget.mkdirs(zone1); + + DFSTestUtil.createKey("test_key", cluster, 0, CONF); + HdfsAdmin hdfsAdmin = new HdfsAdmin(cluster.getURI(0), CONF); + final EnumSet provisionTrash = EnumSet.of(CreateEncryptionZoneFlag.PROVISION_TRASH); + hdfsAdmin.createEncryptionZone(zone1, "test_key", provisionTrash); + + UserGroupInformation ugi = UserGroupInformation.createRemoteUser("foo"); + Path p = (Path) ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = FileSystem.get(FsConstants.VIEWFS_URI, this.conf); + return wFs.getEnclosingRoot(zone); + }); + assertEquals(p, getViewFsPath("/data", fsView)); + p = (Path) ugi.doAs((PrivilegedExceptionAction) () -> { + FileSystem wFs = FileSystem.get(FsConstants.VIEWFS_URI, this.conf); + return wFs.getEnclosingRoot(zone1); + }); + assertEquals(p, getViewFsPath(zone1, fsView)); + + + } finally { + DFSTestUtil.deleteKey("test_key", cluster, 0); + } + } }