From 01a2419e9c942c0a34c40a14d4074a7420cd4420 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Thu, 8 Nov 2018 15:22:39 -0800 Subject: [PATCH 01/12] HADOOP-15891: provide Regex Based Mount Point In Inode Tree Differential Revision: https://phabricator.twitter.biz/D526212 --- .../apache/hadoop/fs/viewfs/ConfigUtil.java | 35 ++ .../apache/hadoop/fs/viewfs/Constants.java | 16 + .../apache/hadoop/fs/viewfs/InodeTree.java | 500 ++++++++++++------ .../hadoop/fs/viewfs/RegexMountPoint.java | 244 +++++++++ .../fs/viewfs/RegexMountPointInterceptor.java | 70 +++ .../RegexMountPointInterceptorFactory.java | 67 +++ .../RegexMountPointInterceptorType.java | 53 ++ ...ointResolvedDstPathReplaceInterceptor.java | 134 +++++ .../hadoop/fs/viewfs/TestRegexMountPoint.java | 156 ++++++ ...TestRegexMountPointInterceptorFactory.java | 52 ++ ...ointResolvedDstPathReplaceInterceptor.java | 99 ++++ .../fs/viewfs/ViewFileSystemBaseTest.java | 49 +- .../hadoop-hdfs/src/site/markdown/ViewFs.md | 76 +++ .../viewfs/TestViewFileSystemLinkRegex.java | 320 +++++++++++ 14 files changed, 1714 insertions(+), 157 deletions(-) create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptor.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorType.java create mode 100644 hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java create mode 100644 hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java create mode 100644 hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java index 7d29b8f44ca62..fdb0a2c204cac 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java @@ -166,6 +166,41 @@ public static void addLinkNfly(final Configuration conf, final String src, addLinkNfly(conf, getDefaultMountTableName(conf), src, null, targets); } + + /** + * Add a LinkRegex to the config for the specified mount table. + * @param conf + * @param mountTableName + * @param srcRegex + * @param targetStr + */ + public static void addLinkRegex( + Configuration conf, final String mountTableName, + final String srcRegex, final String targetStr) { + addLinkRegex(conf, mountTableName, srcRegex, targetStr, null); + } + + /** + * Add a LinkRegex to the config for the specified mount table. + * @param conf + * @param mountTableName + * @param srcRegex + * @param targetStr + * @param interceptorSettings + */ + public static void addLinkRegex( + Configuration conf, final String mountTableName, final String srcRegex, + final String targetStr, final String interceptorSettings) { + String prefix = getConfigViewFsPrefix(mountTableName) + "." + + Constants.CONFIG_VIEWFS_LINK_REGEX + "."; + if ((interceptorSettings != null) && (!interceptorSettings.isEmpty())) { + prefix = prefix + interceptorSettings + + RegexMountPoint.SETTING_SRCREGEX_SEP; + } + String key = prefix + srcRegex; + conf.set(key, targetStr); + } + /** * Add config variable for homedir for default mount table * @param conf - add to this conf diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index 492cb87ee024e..53c55eef0e8e2 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -86,6 +86,14 @@ public interface Constants { */ String CONFIG_VIEWFS_LINK_MERGE_SLASH = "linkMergeSlash"; + /** + * Config variable for specifying a regex link which uses regular expressions + * as source and target could use group captured in src. + * E.g. (^/(?\\w+), /prefix-${firstDir}) => + * (/path1/file1 => /prefix-path1/file1) + */ + String CONFIG_VIEWFS_LINK_REGEX = "linkRegex"; + FsPermission PERMISSION_555 = new FsPermission((short) 0555); String CONFIG_VIEWFS_RENAME_STRATEGY = "fs.viewfs.rename.strategy"; @@ -117,4 +125,12 @@ public interface Constants { "fs.viewfs.ignore.port.in.mount.table.name"; boolean CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME_DEFAULT = false; + + /** + * Config capacity of mount point resolution cache. + * Value <= 0 means disable cache. + */ + String CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY + = "fs.viewfs.path.resolution.cache.capacity"; + int CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY_DEFAULT = 0; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 1f1adea6dbe9c..70adb32e38631 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.fs.viewfs; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.io.FileNotFoundException; import java.io.IOException; @@ -29,7 +30,9 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.concurrent.locks.ReentrantReadWriteLock; +import org.apache.commons.collections.map.LRUMap; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; @@ -39,6 +42,8 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; /** * InodeTree implements a mount-table as a tree of inodes. @@ -46,19 +51,22 @@ * In order to use it the caller must subclass it and implement * the abstract methods {@link #getTargetFileSystem(INodeDir)}, etc. * - * The mountable is initialized from the config variables as + * The mountable is initialized from the config variables as * specified in {@link ViewFs} * * @param is AbstractFileSystem or FileSystem * * The two main methods are * {@link #InodeTree(Configuration, String)} // constructor - * {@link #resolve(String, boolean)} + * {@link #resolve(String, boolean)} */ @InterfaceAudience.Private @InterfaceStability.Unstable abstract class InodeTree { + private static final Logger LOGGER = + LoggerFactory.getLogger(InodeTree.class.getName()); + enum ResultKind { INTERNAL_DIR, EXTERNAL_DIR @@ -72,6 +80,11 @@ enum ResultKind { // the homedir for this mount table private final String homedirPrefix; private List> mountPoints = new ArrayList>(); + private List> regexMountPointList = + new ArrayList>(); + private LRUMap pathResolutionCache; + private ReentrantReadWriteLock cacheRWLock; + private int pathResolutionCacheCapacity; static class MountPoint { String src; @@ -122,7 +135,7 @@ boolean isLink() { */ static class INodeDir extends INode { private final Map> children = new HashMap<>(); - private T internalDirFs = null; //filesystem of this internal directory + private T internalDirFs = null; //filesystem of this internal directory private boolean isRoot = false; private INodeLink fallbackLink = null; @@ -226,7 +239,14 @@ enum LinkType { * Config prefix: fs.viewfs.mounttable..linkNfly * Refer: {@link Constants#CONFIG_VIEWFS_LINK_NFLY} */ - NFLY; + NFLY, + /** + * Link entry which source are regex exrepssions and target refer matched + * group from source + * Config prefix: fs.viewfs.mounttable..linkMerge + * Refer: {@link Constants#CONFIG_VIEWFS_LINK_REGEX} + */ + REGEX; } /** @@ -462,7 +482,7 @@ Configuration getConfig() { /** * Create Inode Tree from the specified mount-table specified in Config - * @param config - the mount table keys are prefixed with + * @param config - the mount table keys are prefixed with * FsConstants.CONFIG_VIEWFS_PREFIX * @param viewName - the name of the mount table - if null use defaultMT name * @throws UnsupportedFileSystemException @@ -495,84 +515,113 @@ protected InodeTree(final Configuration config, final String viewName, final UserGroupInformation ugi = UserGroupInformation.getCurrentUser(); for (Entry si : config) { final String key = si.getKey(); - if (key.startsWith(mountTablePrefix)) { - gotMountTableEntry = true; - LinkType linkType; - String src = key.substring(mountTablePrefix.length()); - String settings = null; - if (src.startsWith(linkPrefix)) { - src = src.substring(linkPrefix.length()); - if (src.equals(SlashPath.toString())) { - throw new UnsupportedFileSystemException("Unexpected mount table " - + "link entry '" + key + "'. Use " - + Constants.CONFIG_VIEWFS_LINK_MERGE_SLASH + " instead!"); - } - linkType = LinkType.SINGLE; - } else if (src.startsWith(linkFallbackPrefix)) { - if (src.length() != linkFallbackPrefix.length()) { - throw new IOException("ViewFs: Mount points initialization error." + - " Invalid " + Constants.CONFIG_VIEWFS_LINK_FALLBACK + - " entry in config: " + src); - } - linkType = LinkType.SINGLE_FALLBACK; - } else if (src.startsWith(linkMergePrefix)) { // A merge link - src = src.substring(linkMergePrefix.length()); - linkType = LinkType.MERGE; - } else if (src.startsWith(linkMergeSlashPrefix)) { - // This is a LinkMergeSlash entry. This entry should - // not have any additional source path. - if (src.length() != linkMergeSlashPrefix.length()) { - throw new IOException("ViewFs: Mount points initialization error." + - " Invalid " + Constants.CONFIG_VIEWFS_LINK_MERGE_SLASH + - " entry in config: " + src); - } - linkType = LinkType.MERGE_SLASH; - } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_NFLY)) { - // prefix.settings.src - src = src.substring(Constants.CONFIG_VIEWFS_LINK_NFLY.length() + 1); - // settings.src - settings = src.substring(0, src.indexOf('.')); - // settings - - // settings.src - src = src.substring(settings.length() + 1); - // src - - linkType = LinkType.NFLY; - } else if (src.startsWith(Constants.CONFIG_VIEWFS_HOMEDIR)) { - // ignore - we set home dir from config - continue; - } else { - throw new IOException("ViewFs: Cannot initialize: Invalid entry in " + - "Mount table in config: " + src); - } + if (!key.startsWith(mountTablePrefix)) { + continue; + } + gotMountTableEntry = true; + LinkType linkType; + String src = key.substring(mountTablePrefix.length()); + String settings = null; + if (src.startsWith(linkPrefix)) { + src = src.substring(linkPrefix.length()); + if (src.equals(SlashPath.toString())) { + throw new UnsupportedFileSystemException("Unexpected mount table " + + "link entry '" + key + "'. Use " + + Constants.CONFIG_VIEWFS_LINK_MERGE_SLASH + " instead!"); + } + linkType = LinkType.SINGLE; + } else if (src.startsWith(linkFallbackPrefix)) { + if (src.length() != linkFallbackPrefix.length()) { + throw new IOException("ViewFs: Mount points initialization error." + + " Invalid " + Constants.CONFIG_VIEWFS_LINK_FALLBACK + + " entry in config: " + src); + } + linkType = LinkType.SINGLE_FALLBACK; + } else if (src.startsWith(linkMergePrefix)) { // A merge link + src = src.substring(linkMergePrefix.length()); + linkType = LinkType.MERGE; + } else if (src.startsWith(linkMergeSlashPrefix)) { + // This is a LinkMergeSlash entry. This entry should + // not have any additional source path. + if (src.length() != linkMergeSlashPrefix.length()) { + throw new IOException("ViewFs: Mount points initialization error." + + " Invalid " + Constants.CONFIG_VIEWFS_LINK_MERGE_SLASH + + " entry in config: " + src); + } + linkType = LinkType.MERGE_SLASH; + } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_NFLY)) { + // prefix.settings.src + src = src.substring(Constants.CONFIG_VIEWFS_LINK_NFLY.length() + 1); + // settings.src + settings = src.substring(0, src.indexOf('.')); + // settings + + // settings.src + src = src.substring(settings.length() + 1); + // src + + linkType = LinkType.NFLY; + } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_REGEX)) { final String target = si.getValue(); - if (linkType != LinkType.MERGE_SLASH) { - if (isMergeSlashConfigured) { - throw new IOException("Mount table " + mountTableName - + " has already been configured with a merge slash link. " - + "A regular link should not be added."); - } - linkEntries.add( - new LinkEntry(src, target, linkType, settings, ugi, config)); + String linkKeyPath = null; + final String linkRegexPrefix = Constants.CONFIG_VIEWFS_LINK_REGEX + "."; + // settings#.linkKey + String settingsAndLinkKeyPath = src.substring(linkRegexPrefix.length()); + int settingLinkKeySepIndex = settingsAndLinkKeyPath + .indexOf(RegexMountPoint.SETTING_SRCREGEX_SEP); + if (settingLinkKeySepIndex == -1) { + // There's no settings + linkKeyPath = settingsAndLinkKeyPath; + settings = null; } else { - if (!linkEntries.isEmpty()) { - throw new IOException("Mount table " + mountTableName - + " has already been configured with regular links. " - + "A merge slash link should not be configured."); - } - if (isMergeSlashConfigured) { - throw new IOException("Mount table " + mountTableName - + " has already been configured with a merge slash link. " - + "Multiple merge slash links for the same mount table is " - + "not allowed."); - } - isMergeSlashConfigured = true; - mergeSlashTarget = target; + // settings#.linkKey style configuration + // settings from settings#.linkKey + settings = + settingsAndLinkKeyPath.substring(0, settingLinkKeySepIndex); + // linkKeyPath + linkKeyPath = settingsAndLinkKeyPath.substring( + settings.length() + RegexMountPoint.SETTING_SRCREGEX_SEP + .length()); } + linkType = LinkType.REGEX; + linkEntries.add( + new LinkEntry(linkKeyPath, target, linkType, settings, ugi, + config)); + continue; + } else if (src.startsWith(Constants.CONFIG_VIEWFS_HOMEDIR)) { + // ignore - we set home dir from config + continue; + } else { + throw new IOException("ViewFs: Cannot initialize: Invalid entry in " + + "Mount table in config: " + src); } - } + + final String target = si.getValue(); + if (linkType != LinkType.MERGE_SLASH) { + if (isMergeSlashConfigured) { + throw new IOException("Mount table " + mountTableName + + " has already been configured with a merge slash link. " + + "A regular link should not be added."); + } + linkEntries.add( + new LinkEntry(src, target, linkType, settings, ugi, config)); + } else { + if (!linkEntries.isEmpty()) { + throw new IOException("Mount table " + mountTableName + + " has already been configured with regular links. " + + "A merge slash link should not be configured."); + } + if (isMergeSlashConfigured) { + throw new IOException("Mount table " + mountTableName + + " has already been configured with a merge slash link. " + + "Multiple merge slash links for the same mount table is " + + "not allowed."); + } + isMergeSlashConfigured = true; + mergeSlashTarget = target; + } + } // End of for loop. if (isMergeSlashConfigured) { Preconditions.checkNotNull(mergeSlashTarget); @@ -587,7 +636,8 @@ protected InodeTree(final Configuration config, final String viewName, getRootDir().setRoot(true); INodeLink fallbackLink = null; for (LinkEntry le : linkEntries) { - if (le.isLinkType(LinkType.SINGLE_FALLBACK)) { + switch (le.getLinkType()) { + case SINGLE_FALLBACK: if (fallbackLink != null) { throw new IOException("Mount table " + mountTableName + " has already been configured with a link fallback. " @@ -597,7 +647,18 @@ protected InodeTree(final Configuration config, final String viewName, fallbackLink = new INodeLink(mountTableName, ugi, getTargetFileSystem(new URI(le.getTarget())), new URI(le.getTarget())); - } else { + continue; + case REGEX: + LOGGER.info("Add regex mount point:" + le.getSrc() + + ", target:" + le.getTarget() + + ", interceptor settings:" + le.getSettings()); + RegexMountPoint regexMountPoint = + new RegexMountPoint( + this, le.getSrc(), le.getTarget(), le.getSettings()); + regexMountPoint.initialize(); + regexMountPointList.add(regexMountPoint); + continue; + default: createLink(le.getSrc(), le.getTarget(), le.getLinkType(), le.getSettings(), le.getUgi(), le.getConfig()); } @@ -622,6 +683,13 @@ protected InodeTree(final Configuration config, final String viewName, theUri); getRootDir().addFallbackLink(rootFallbackLink); } + pathResolutionCacheCapacity = config + .getInt(Constants.CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY, + Constants.CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY_DEFAULT); + if (pathResolutionCacheCapacity > 0) { + pathResolutionCache = new LRUMap(pathResolutionCacheCapacity); + cacheRWLock = new ReentrantReadWriteLock(); + } } /** @@ -656,7 +724,7 @@ boolean isInternalDir() { } /** - * Resolve the pathname p relative to root InodeDir + * Resolve the pathname p relative to root InodeDir. * @param p - input path * @param resolveLastComponent * @return ResolveResult which allows further resolution of the remaining path @@ -664,94 +732,214 @@ boolean isInternalDir() { */ ResolveResult resolve(final String p, final boolean resolveLastComponent) throws FileNotFoundException { - String[] path = breakIntoPathComponents(p); - if (path.length <= 1) { // special case for when path is "/" - T targetFs = root.isInternalDir() ? - getRootDir().getInternalDirFs() : getRootLink().getTargetFileSystem(); - ResolveResult res = new ResolveResult(ResultKind.INTERNAL_DIR, - targetFs, root.fullPath, SlashPath); - return res; - } + ResolveResult resolveResult = null; + resolveResult = getResolveResultFromCache(p, resolveLastComponent); + if (resolveResult != null) { + return resolveResult; + } + + try { + String[] path = breakIntoPathComponents(p); + if (path.length <= 1) { // special case for when path is "/" + T targetFs = root.isInternalDir() ? + getRootDir().getInternalDirFs() + : getRootLink().getTargetFileSystem(); + resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, + targetFs, root.fullPath, SlashPath); + return resolveResult; + } - /** - * linkMergeSlash has been configured. The root of this mount table has - * been linked to the root directory of a file system. - * The first non-slash path component should be name of the mount table. - */ - if (root.isLink()) { - Path remainingPath; - StringBuilder remainingPathStr = new StringBuilder(); - // ignore first slash - for (int i = 1; i < path.length; i++) { - remainingPathStr.append("/").append(path[i]); + /** + * linkMergeSlash has been configured. The root of this mount table has + * been linked to the root directory of a file system. + * The first non-slash path component should be name of the mount table. + */ + if (root.isLink()) { + Path remainingPath; + StringBuilder remainingPathStr = new StringBuilder(); + // ignore first slash + for (int i = 1; i < path.length; i++) { + remainingPathStr.append("/").append(path[i]); + } + remainingPath = new Path(remainingPathStr.toString()); + resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, + getRootLink().getTargetFileSystem(), root.fullPath, remainingPath); + return resolveResult; } - remainingPath = new Path(remainingPathStr.toString()); - ResolveResult res = new ResolveResult(ResultKind.EXTERNAL_DIR, - getRootLink().getTargetFileSystem(), root.fullPath, remainingPath); - return res; - } - Preconditions.checkState(root.isInternalDir()); - INodeDir curInode = getRootDir(); + Preconditions.checkState(root.isInternalDir()); + INodeDir curInode = getRootDir(); - int i; - // ignore first slash - for (i = 1; i < path.length - (resolveLastComponent ? 0 : 1); i++) { - INode nextInode = curInode.resolveInternal(path[i]); - if (nextInode == null) { - if (hasFallbackLink()) { - return new ResolveResult(ResultKind.EXTERNAL_DIR, - getRootFallbackLink().getTargetFileSystem(), - root.fullPath, new Path(p)); - } else { - StringBuilder failedAt = new StringBuilder(path[0]); - for (int j = 1; j <= i; ++j) { - failedAt.append('/').append(path[j]); + // Try to resolve path in the regex mount point + resolveResult = tryResolveInRegexMountpoint(p, resolveLastComponent); + if (resolveResult != null) { + return resolveResult; + } + + int i; + // ignore first slash + for (i = 1; i < path.length - (resolveLastComponent ? 0 : 1); i++) { + INode nextInode = curInode.resolveInternal(path[i]); + if (nextInode == null) { + if (hasFallbackLink()) { + resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, + getRootFallbackLink().getTargetFileSystem(), root.fullPath, + new Path(p)); + return resolveResult; + } else { + StringBuilder failedAt = new StringBuilder(path[0]); + for (int j = 1; j <= i; ++j) { + failedAt.append('/').append(path[j]); + } + throw (new FileNotFoundException( + "File/Directory does not exist: " + failedAt.toString())); } - throw (new FileNotFoundException( - "File/Directory does not exist: " + failedAt.toString())); } - } - if (nextInode.isLink()) { - final INodeLink link = (INodeLink) nextInode; - final Path remainingPath; - if (i >= path.length - 1) { - remainingPath = SlashPath; - } else { - StringBuilder remainingPathStr = new StringBuilder("/" + path[i + 1]); - for (int j = i + 2; j < path.length; ++j) { - remainingPathStr.append('/').append(path[j]); + if (nextInode.isLink()) { + final INodeLink link = (INodeLink) nextInode; + final Path remainingPath; + if (i >= path.length - 1) { + remainingPath = SlashPath; + } else { + StringBuilder remainingPathStr = + new StringBuilder("/" + path[i + 1]); + for (int j = i + 2; j < path.length; ++j) { + remainingPathStr.append('/').append(path[j]); + } + remainingPath = new Path(remainingPathStr.toString()); } - remainingPath = new Path(remainingPathStr.toString()); + resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, + link.getTargetFileSystem(), nextInode.fullPath, remainingPath); + return resolveResult; + } else if (nextInode.isInternalDir()) { + curInode = (INodeDir) nextInode; } - final ResolveResult res = - new ResolveResult(ResultKind.EXTERNAL_DIR, - link.getTargetFileSystem(), nextInode.fullPath, remainingPath); - return res; - } else if (nextInode.isInternalDir()) { - curInode = (INodeDir) nextInode; + } + + // We have resolved to an internal dir in mount table. + Path remainingPath; + if (resolveLastComponent) { + remainingPath = SlashPath; + } else { + // note we have taken care of when path is "/" above + // for internal dirs rem-path does not start with / since the lookup + // that follows will do a children.get(remaningPath) and will have to + // strip-out the initial / + StringBuilder remainingPathStr = new StringBuilder("/" + path[i]); + for (int j = i + 1; j < path.length; ++j) { + remainingPathStr.append('/').append(path[j]); + } + remainingPath = new Path(remainingPathStr.toString()); + } + resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, + curInode.getInternalDirFs(), curInode.fullPath, remainingPath); + return resolveResult; + } finally { + if (pathResolutionCacheCapacity > 0 && resolveResult != null) { + addResolveResultToCache(p, resolveLastComponent, resolveResult); } } + } - // We have resolved to an internal dir in mount table. - Path remainingPath; - if (resolveLastComponent) { - remainingPath = SlashPath; - } else { - // note we have taken care of when path is "/" above - // for internal dirs rem-path does not start with / since the lookup - // that follows will do a children.get(remaningPath) and will have to - // strip-out the initial / - StringBuilder remainingPathStr = new StringBuilder("/" + path[i]); - for (int j = i + 1; j < path.length; ++j) { - remainingPathStr.append('/').append(path[j]); + /** + * Walk through all regex mount points to see + * whether the path match any regex expressions. + * + * @param srcPath + * @param resolveLastComponent + * @return + */ + protected ResolveResult tryResolveInRegexMountpoint(final String srcPath, + final boolean resolveLastComponent) { + for (RegexMountPoint regexMountPoint : regexMountPointList) { + ResolveResult resolveResult = + regexMountPoint.resolve(srcPath, resolveLastComponent); + if (resolveResult != null) { + return resolveResult; } - remainingPath = new Path(remainingPathStr.toString()); } - final ResolveResult res = - new ResolveResult(ResultKind.INTERNAL_DIR, - curInode.getInternalDirFs(), curInode.fullPath, remainingPath); - return res; + return null; + } + + /** + * Build resolve result return to caller. + * + * @param resultKind + * @param resolvedPathStr + * @param targetOfResolvedPathStr + * @param remainingPath + * @return + */ + protected ResolveResult buildResolveResultForRegexMountPoint( + ResultKind resultKind, String resolvedPathStr, + String targetOfResolvedPathStr, Path remainingPath) { + try { + T targetFs = getTargetFileSystem(new URI(targetOfResolvedPathStr)); + return new ResolveResult(resultKind, targetFs, resolvedPathStr, + remainingPath); + } catch (IOException ex) { + LOGGER.error(String.format( + "Got Exception while build resolve result." + + " ResultKind:%s, resolvedPathStr:%s," + + " targetOfResolvedPathStr:%s, remainingPath:%s," + + " will return null.", + resultKind, resolvedPathStr, targetOfResolvedPathStr, remainingPath), + ex); + return null; + } catch (URISyntaxException uex) { + LOGGER.error(String.format( + "Got Exception while build resolve result." + + " ResultKind:%s, resolvedPathStr:%s," + + " targetOfResolvedPathStr:%s, remainingPath:%s," + + " will return null.", + resultKind, resolvedPathStr, targetOfResolvedPathStr, remainingPath), + uex); + return null; + } + } + + /** + * Return resolution cache capacity. + * + * @return + */ + public int getPathResolutionCacheCapacity() { + return pathResolutionCacheCapacity; + } + + private void addResolveResultToCache(final String pathStr, + final Boolean resolveLastComponent, + final ResolveResult resolveResult) { + try { + cacheRWLock.writeLock().lock(); + String key = getResolveCacheKeyStr(pathStr, resolveLastComponent); + pathResolutionCache.put(key, resolveResult); + } finally { + cacheRWLock.writeLock().unlock(); + } + } + + private ResolveResult getResolveResultFromCache(final String pathStr, + final Boolean resolveLastComponent) { + if (pathResolutionCacheCapacity <= 0) { + return null; + } + try { + cacheRWLock.readLock().lock(); + String key = getResolveCacheKeyStr(pathStr, resolveLastComponent); + return (ResolveResult) pathResolutionCache.get(key); + } finally { + cacheRWLock.readLock().unlock(); + } + } + + public static String getResolveCacheKeyStr(final String path, + Boolean resolveLastComp) { + return path + ",resolveLastComp" + resolveLastComp; + } + + @VisibleForTesting public LRUMap getPathResolutionCache() { + return pathResolutionCache; } List> getMountPoints() { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java new file mode 100644 index 0000000000000..5d8049b049658 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java @@ -0,0 +1,244 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.util.StringUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.fs.viewfs.InodeTree.SlashPath; + +/** + * Regex mount point is build to implement regex based mount point. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class RegexMountPoint { + private static final Logger LOGGER = + LoggerFactory.getLogger(RegexMountPoint.class.getName()); + + private InodeTree inodeTree; + private String srcPathRegex; + private Pattern srcPattern; + private String dstPath; + private String interceptorSettingsString; + private List interceptorList; + + public static final String SETTING_SRCREGEX_SEP = "#."; + public static final char INTERCEPTOR_SEP = ';'; + public static final char INTERCEPTOR_INTERNAL_SEP = ':'; + // ${var},$var + public static final Pattern VAR_PATTERN_IN_DEST = + Pattern.compile("\\$((\\{\\w+\\})|(\\w+))"); + + // key => $key or key = > ${key} + private Map> varInDestPathMap; + + public Map> getVarInDestPathMap() { + return varInDestPathMap; + } + + RegexMountPoint(InodeTree inodeTree, String sourcePathRegex, + String destPath, String settingsStr) { + this.inodeTree = inodeTree; + this.srcPathRegex = sourcePathRegex; + this.dstPath = destPath; + this.interceptorSettingsString = settingsStr; + this.interceptorList = new ArrayList<>(); + } + + /** + * Initialize regex mount point. + * + * @throws IOException + */ + public void initialize() throws IOException { + try { + srcPattern = Pattern.compile(srcPathRegex); + } catch (PatternSyntaxException ex) { + throw new IOException( + "Failed to initialized mount point due to bad src path regex:" + + srcPathRegex + ", dstPath:" + dstPath, ex); + } + varInDestPathMap = getVarListInString(dstPath); + initializeInterceptors(); + } + + private void initializeInterceptors() throws IOException { + if (interceptorSettingsString == null + || interceptorSettingsString.isEmpty()) { + return; + } + String[] interceptorStrArray = + StringUtils.split(interceptorSettingsString, INTERCEPTOR_SEP); + for (String interceptorStr : interceptorStrArray) { + RegexMountPointInterceptor interceptor = + RegexMountPointInterceptorFactory.create(interceptorStr); + if (interceptor == null) { + throw new IOException( + "Illegal settings String " + interceptorSettingsString); + } + interceptor.initialize(); + interceptorList.add(interceptor); + } + } + + /** + * Get $var1 and $var2 style variables in string. + * + * @param input + * @return + */ + public static Map> getVarListInString(String input) { + Map> varMap = new HashMap<>(); + Matcher matcher = VAR_PATTERN_IN_DEST.matcher(input); + while (matcher.find()) { + // $var or ${var} + String varName = matcher.group(0); + // var or {var} + String strippedVarName = matcher.group(1); + if (strippedVarName.startsWith("{")) { + // {varName} = > varName + strippedVarName = + strippedVarName.substring(1, strippedVarName.length() - 1); + } + varMap.putIfAbsent(strippedVarName, new HashSet<>()); + varMap.get(strippedVarName).add(varName); + } + return varMap; + } + + public String getSrcPathRegex() { + return srcPathRegex; + } + + public Pattern getSrcPattern() { + return srcPattern; + } + + public String getDstPath() { + return dstPath; + } + + public static Pattern getVarPatternInDest() { + return VAR_PATTERN_IN_DEST; + } + + /** + * Get resolved path from regex mount points. + * @param srcPath + * @param resolveLastComponent + * @return + */ + public InodeTree.ResolveResult resolve(final String srcPath, + final boolean resolveLastComponent) { + String pathStrToResolve = srcPath; + if (!resolveLastComponent) { + int lastSlashIndex = srcPath.lastIndexOf(SlashPath.toString()); + if (lastSlashIndex == -1) { + return null; + } + pathStrToResolve = srcPath.substring(0, lastSlashIndex); + } + for (RegexMountPointInterceptor interceptor : interceptorList) { + pathStrToResolve = interceptor.interceptSource(pathStrToResolve); + } + LOGGER.debug("Path to resolve:" + pathStrToResolve + ",srcPattern:" + + getSrcPathRegex()); + Matcher srcMatcher = getSrcPattern().matcher(pathStrToResolve); + String parsedDestPath = getDstPath(); + int mappedCount = 0; + String resolvedPathStr = ""; + while (srcMatcher.find()) { + resolvedPathStr = pathStrToResolve.substring(0, srcMatcher.end()); + Map> varMap = getVarInDestPathMap(); + for (Map.Entry> entry : varMap.entrySet()) { + String groupNameOrIndexStr = entry.getKey(); + String groupValue = null; + if (groupNameOrIndexStr.matches("\\d+")) { + // group index + int groupIndex = Integer.parseUnsignedInt(groupNameOrIndexStr); + if (groupIndex >= 0 && groupIndex <= srcMatcher.groupCount()) { + groupValue = srcMatcher.group(groupIndex); + } + } else { + // named group in regex + groupValue = srcMatcher.group(groupNameOrIndexStr); + } + if (groupValue == null) { + continue; + } + Set varNameListToReplace = entry.getValue(); + for (String varName : varNameListToReplace) { + parsedDestPath = parsedDestPath.replace(varName, groupValue); + LOGGER.debug("parsedDestPath value is:" + parsedDestPath); + } + } + ++mappedCount; + } + if (0 == mappedCount) { + return null; + } + String remainingPathStr = srcPath.substring(resolvedPathStr.length()); + if (!remainingPathStr.startsWith("/")) { + remainingPathStr = "/" + remainingPathStr; + } + Path remainingPath = new Path(remainingPathStr); + for (RegexMountPointInterceptor interceptor : interceptorList) { + parsedDestPath = interceptor.interceptResolvedDestPathStr(parsedDestPath); + remainingPath = + interceptor.interceptRemainingPath(remainingPath); + } + InodeTree.ResolveResult resolveResult = inodeTree + .buildResolveResultForRegexMountPoint(InodeTree.ResultKind.EXTERNAL_DIR, + resolvedPathStr, parsedDestPath, remainingPath); + return resolveResult; + } + + /** + * Convert interceptor to string. + * + * @param interceptorList + * @return + */ + public static String convertInterceptorsToString( + List interceptorList) { + StringBuffer stringBuffer = new StringBuffer(); + for (int index = 0; index < interceptorList.size(); ++index) { + stringBuffer.append(interceptorList.get(index).toString()); + if (index < interceptorList.size() - 1) { + stringBuffer.append(INTERCEPTOR_SEP); + } + } + return stringBuffer.toString(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptor.java new file mode 100644 index 0000000000000..37f44b0a51579 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptor.java @@ -0,0 +1,70 @@ +/** + * 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.viewfs; + +import java.io.IOException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +/** + * RegexMountPointInterceptor is a mechanism provided to intercept + * src and resolved path before/after resolution. + */ +@InterfaceAudience.LimitedPrivate("Common") +@InterfaceStability.Unstable +interface RegexMountPointInterceptor { + + /** + * Initialize interceptor and throws IOException if needed. + * @throws IOException + */ + void initialize() throws IOException; + + /** + * Intercept source before resolution. + * @param source + * @return + */ + String interceptSource(String source); + + /** + * Intercept parsed dest path and return a new one. + * @return intercepted string + */ + String interceptResolvedDestPathStr(String parsedDestPathStr); + + /** + * Intercept remaining path. + * @return intercepted string + */ + Path interceptRemainingPath(Path remainingPath); + + /** + * Get interceptor type. + * @return + */ + RegexMountPointInterceptorType getType(); + + /** + * Serialize the interceptor to a string. + * @return + */ + String serializeToString(); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java new file mode 100644 index 0000000000000..d9b3f54021333 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java @@ -0,0 +1,67 @@ +/** + * 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.viewfs; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * The interceptor factory used to create RegexMountPoint interceptors. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +final class RegexMountPointInterceptorFactory { + + private RegexMountPointInterceptorFactory() { + + } + + /** + * interceptorSettingsString string should be like ${type}:${string}, + * e.g. replaceresolveddstpath:word1,word2. + * + * @param interceptorSettingsString + * @return + */ + public static RegexMountPointInterceptor create( + String interceptorSettingsString) { + int typeTagIndex = interceptorSettingsString + .indexOf(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP); + if (typeTagIndex == -1 || (typeTagIndex == ( + interceptorSettingsString.length() - 1))) { + return null; + } + String typeTag = interceptorSettingsString.substring(0, typeTagIndex).trim() + .toLowerCase(); + RegexMountPointInterceptorType interceptorType = + RegexMountPointInterceptorType.get(typeTag); + if (interceptorType == null) { + return null; + } + switch (interceptorType) { + case REPLACE_RESOLVED_DST_PATH: + RegexMountPointInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(interceptorSettingsString); + return interceptor; + default: + // impossible now + return null; + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorType.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorType.java new file mode 100644 index 0000000000000..ad953eba24ad9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorType.java @@ -0,0 +1,53 @@ +/** + * 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.viewfs; + +import java.util.HashMap; +import java.util.Map; + +/** + * RegexMountPointInterceptorType. + */ +public enum RegexMountPointInterceptorType { + REPLACE_RESOLVED_DST_PATH("replaceresolveddstpath"); + + private final String configName; + private static final Map + INTERCEPTOR_TYPE_MAP + = new HashMap(); + + static { + for (RegexMountPointInterceptorType interceptorType + : RegexMountPointInterceptorType.values()) { + INTERCEPTOR_TYPE_MAP.put( + interceptorType.getConfigName(), interceptorType); + } + } + + RegexMountPointInterceptorType(String configName) { + this.configName = configName; + } + + public String getConfigName() { + return configName; + } + + public static RegexMountPointInterceptorType get(String configName) { + return INTERCEPTOR_TYPE_MAP.get(configName); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java new file mode 100644 index 0000000000000..d89b93df2b973 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java @@ -0,0 +1,134 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.util.regex.Matcher; +import java.util.regex.Pattern; +import java.util.regex.PatternSyntaxException; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Path; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Implementation of RegexMountPointResolvedDstPathReplaceInterceptor. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class RegexMountPointResolvedDstPathReplaceInterceptor + implements RegexMountPointInterceptor { + + private String srcRegexString; + private String replaceString; + private Pattern srcRegexPattern; + + RegexMountPointResolvedDstPathReplaceInterceptor(String srcRegex, + String replaceString) { + this.srcRegexString = srcRegex; + this.replaceString = replaceString; + this.srcRegexPattern = null; + } + + public String getSrcRegexString() { + return srcRegexString; + } + + public String getReplaceString() { + return replaceString; + } + + public Pattern getSrcRegexPattern() { + return srcRegexPattern; + } + + @Override public void initialize() throws IOException { + try { + srcRegexPattern = Pattern.compile(srcRegexString); + } catch (PatternSyntaxException ex) { + throw new IOException( + "Initialize interceptor failed, srcRegx:" + srcRegexString, ex); + } + } + + /** + * Intercept source before resolution. + * + * @param source + * @return + */ + @Override public String interceptSource(String source) { + return source; + } + + /** + * Intercept resolved path, e.g. + * Mount point /^(\\w+)/, ${1}.hadoop.net + * If incoming path is /user1/home/tmp/job1, + * then the resolved path str will be user1. + * + * @return intercepted string + */ + @Override public String interceptResolvedDestPathStr( + String parsedDestPathStr) { + Matcher matcher = srcRegexPattern.matcher(parsedDestPathStr); + return matcher.replaceAll(replaceString); + } + + /** + * Intercept remaining path. + * + * @return intercepted path + */ + @Override public Path interceptRemainingPath(Path remainingPath) { + return remainingPath; + } + + @Override public RegexMountPointInterceptorType getType() { + return REPLACE_RESOLVED_DST_PATH; + } + + @Override public String serializeToString() { + return REPLACE_RESOLVED_DST_PATH.getConfigName() + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + srcRegexString + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + replaceString; + } + + /** + * Create interceptor from config string. The string should be in + * replaceresolvedpath:wordToReplace:replaceString + * Note that we'll assume there's no ':' in the regex for the moment. + * + * @param serializedString + * @return + */ + public static RegexMountPointResolvedDstPathReplaceInterceptor + deserializeFromString(String serializedString) { + String[] strings = serializedString + .split(Character.toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP)); + // We'll assume there's no ':' in the regex for the moment. + if (strings.length != 3) { + return null; + } + //The format should be like replaceresolvedpath:wordToReplace:replaceString + return new RegexMountPointResolvedDstPathReplaceInterceptor(strings[1], + strings[2]); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java new file mode 100644 index 0000000000000..760df4b84650f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java @@ -0,0 +1,156 @@ +/** + * 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.viewfs; + +import java.io.IOException; +import java.net.URI; +import java.util.Map; +import java.util.Set; + +import org.apache.hadoop.conf.Configuration; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Test Regex Mount Point. + */ +public class TestRegexMountPoint { + private static final Logger LOGGER = + LoggerFactory.getLogger(TestRegexMountPoint.class.getName()); + + private InodeTree inodeTree; + private Configuration conf; + + class TestRegexMountPointFileSystem { + public URI getUri() { + return uri; + } + + private URI uri; + + TestRegexMountPointFileSystem(URI uri) { + String uriStr = uri == null ? "null" : uri.toString(); + LOGGER.info("Create TestRegexMountPointFileSystem Via URI:" + uriStr); + this.uri = uri; + } + } + + @Before + public void setUp() throws Exception { + conf = new Configuration(); + ConfigUtil.addLink(conf, TestRegexMountPoint.class.getName(), "/mnt", + URI.create("file:///")); + + inodeTree = new InodeTree(conf, + TestRegexMountPoint.class.getName(), null, false) { + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final URI uri) { + return new TestRegexMountPointFileSystem(uri); + } + + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final INodeDir dir) { + return new TestRegexMountPointFileSystem(null); + } + + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final String settings, final URI[] mergeFsURIList) { + return new TestRegexMountPointFileSystem(null); + } + }; + } + + @After public void tearDown() throws Exception { + inodeTree = null; + } + + @Test public void testGetVarListInString() throws IOException { + String srcRegex = "/(\\w+)"; + String target = "/$0/${1}/$1/${2}/${2}"; + RegexMountPoint regexMountPoint = + new RegexMountPoint(inodeTree, srcRegex, target, null); + regexMountPoint.initialize(); + Map> varMap = regexMountPoint.getVarInDestPathMap(); + Assert.assertEquals(varMap.size(), 3); + Assert.assertEquals(varMap.get("0").size(), 1); + Assert.assertTrue(varMap.get("0").contains("$0")); + Assert.assertEquals(varMap.get("1").size(), 2); + Assert.assertTrue(varMap.get("1").contains("${1}")); + Assert.assertTrue(varMap.get("1").contains("$1")); + Assert.assertEquals(varMap.get("2").size(), 1); + Assert.assertTrue(varMap.get("2").contains("${2}")); + } + + @Test public void testResolve() throws IOException { + String regexStr = "^/user/(?\\w+)"; + String dstPathStr = "/namenode1/testResolve/$username"; + String settingsStr = null; + RegexMountPoint regexMountPoint = + new RegexMountPoint(inodeTree, regexStr, dstPathStr, settingsStr); + regexMountPoint.initialize(); + InodeTree.ResolveResult resolveResult = + regexMountPoint.resolve("/user/hadoop/file1", true); + Assert.assertEquals(resolveResult.kind, InodeTree.ResultKind.EXTERNAL_DIR); + Assert.assertTrue( + resolveResult.targetFileSystem + instanceof TestRegexMountPointFileSystem); + Assert.assertTrue(resolveResult.resolvedPath.equals("/user/hadoop")); + Assert.assertTrue( + resolveResult.targetFileSystem + instanceof TestRegexMountPointFileSystem); + Assert.assertTrue( + ((TestRegexMountPointFileSystem) resolveResult.targetFileSystem) + .getUri().toString().equals("/namenode1/testResolve/hadoop")); + Assert.assertTrue(resolveResult.remainingPath.toString().equals("/file1")); + } + + @Test public void testResolveWithInterceptor() throws IOException { + String regexStr = "^/user/(?\\w+)"; + String dstPathStr = "/namenode1/testResolve/$username"; + // Replace "_" with "-" + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor("_", "-"); + // replaceresolvedpath:_:- + String settingsStr = interceptor.serializeToString(); + RegexMountPoint regexMountPoint = + new RegexMountPoint(inodeTree, regexStr, dstPathStr, settingsStr); + regexMountPoint.initialize(); + InodeTree.ResolveResult resolveResult = + regexMountPoint.resolve("/user/hadoop_user1/file_index", true); + Assert.assertEquals(resolveResult.kind, InodeTree.ResultKind.EXTERNAL_DIR); + Assert.assertTrue( + resolveResult.targetFileSystem + instanceof TestRegexMountPointFileSystem); + Assert.assertTrue(resolveResult.resolvedPath.equals("/user/hadoop_user1")); + Assert.assertTrue( + resolveResult.targetFileSystem + instanceof TestRegexMountPointFileSystem); + Assert.assertTrue( + ((TestRegexMountPointFileSystem) resolveResult.targetFileSystem) + .getUri().toString().equals("/namenode1/testResolve/hadoop-user1")); + Assert.assertTrue( + resolveResult.remainingPath.toString().equals("/file_index")); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java new file mode 100644 index 0000000000000..0b12332353067 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java @@ -0,0 +1,52 @@ +/** + * 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.viewfs; + +import org.junit.Assert; +import org.junit.Test; + +/** + * Test Regex Mount Point Interceptor Factory. + */ +public class TestRegexMountPointInterceptorFactory { + + @Test public void testCreateNormalCase() { + String replaceInterceptorStr = + RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH.getConfigName() + + Character.toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + + "src" + Character + .toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + "replace"; + RegexMountPointInterceptor interceptor = + RegexMountPointInterceptorFactory.create(replaceInterceptorStr); + Assert.assertTrue( + interceptor + instanceof RegexMountPointResolvedDstPathReplaceInterceptor); + } + + @Test public void testCreateBadCase() { + String replaceInterceptorStr = + RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH.getConfigName() + + "___" + Character + .toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + "src" + + Character.toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) + + "replace"; + RegexMountPointInterceptor interceptor = + RegexMountPointInterceptorFactory.create(replaceInterceptorStr); + Assert.assertTrue(interceptor == null); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java new file mode 100644 index 0000000000000..5c44fbb25c089 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java @@ -0,0 +1,99 @@ +/** + * 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.viewfs; + +import java.io.IOException; + +import org.apache.hadoop.fs.Path; +import org.junit.Assert; +import org.junit.Test; + +import static org.apache.hadoop.fs.viewfs.RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH; + +/** + * Test RegexMountPointResolvedDstPathReplaceInterceptor. + */ +public class TestRegexMountPointResolvedDstPathReplaceInterceptor { + + public String createSerializedString(String regex, String replaceString) { + return REPLACE_RESOLVED_DST_PATH.getConfigName() + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + regex + + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + replaceString; + } + + @Test public void testDeserializeFromStringNormalCase() throws IOException { + String srcRegex = "-"; + String replaceString = "_"; + String serializedString = createSerializedString(srcRegex, replaceString); + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(serializedString); + Assert.assertTrue(interceptor.getSrcRegexString().equals(srcRegex)); + Assert.assertTrue(interceptor.getReplaceString().equals(replaceString)); + Assert.assertTrue(interceptor.getSrcRegexPattern() == null); + interceptor.initialize(); + Assert.assertTrue( + interceptor.getSrcRegexPattern().toString().equals(srcRegex)); + } + + @Test public void testDeserializeFromStringBadCase() throws IOException { + String srcRegex = "-"; + String replaceString = "_"; + String serializedString = createSerializedString(srcRegex, replaceString); + serializedString = serializedString + ":ddd"; + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + RegexMountPointResolvedDstPathReplaceInterceptor + .deserializeFromString(serializedString); + Assert.assertEquals(interceptor, null); + } + + @Test public void testSerialization() { + String srcRegex = "word1"; + String replaceString = "word2"; + String serializedString = createSerializedString(srcRegex, replaceString); + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, + replaceString); + Assert.assertEquals(interceptor.serializeToString(), serializedString); + } + + @Test public void testInterceptSource() { + String srcRegex = "word1"; + String replaceString = "word2"; + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, + replaceString); + String sourcePath = "/a/b/l3/dd"; + sourcePath = interceptor.interceptSource(sourcePath); + } + + @Test public void testInterceptResolve() throws IOException { + String pathAfterResolution = "/user-hadoop"; + Path remainingPath = new Path("/ad-data"); + + String srcRegex = "hadoop"; + String replaceString = "hdfs"; + RegexMountPointResolvedDstPathReplaceInterceptor interceptor = + new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, + replaceString); + interceptor.initialize(); + Assert.assertTrue( + interceptor.interceptResolvedDestPathStr(pathAfterResolution) + .equals("/user-hdfs")); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index 05d7974395013..e6b4c53543b83 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -31,6 +31,7 @@ import java.util.Map.Entry; import java.util.Random; +import org.apache.commons.collections.map.LRUMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.BlockStoragePolicySpi; @@ -1417,7 +1418,8 @@ public void testGetContentSummaryWithFileInLocalFS() throws Exception { fos.write(expected.getBytes()); } ConfigUtil.addLink(conf, - "/internalDir/internalDir2/linkToLocalFile", localFile.toURI()); + "/internalDir/internalDir2/linkToLocalFile", + localFile.toURI()); try (FileSystem fs = FileSystem.get(FsConstants.VIEWFS_URI, conf)) { ContentSummary summaryAfter = @@ -1430,4 +1432,49 @@ public void testGetContentSummaryWithFileInLocalFS() throws Exception { summaryAfter.getLength()); } } + + @Test + public void testMountPointCache() throws Exception { + conf.setInt(Constants.CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY, 1); + conf.setBoolean("fs.viewfs.impl.disable.cache", true); + FileSystem fileSystem = FileSystem.get(FsConstants.VIEWFS_URI, conf); + ViewFileSystem viewfs = (ViewFileSystem) fileSystem; + Path resolvedPath1 = new Path(targetTestRoot, "dir3/file1"); + Path srcPath1 = new Path("/internalDir/internalDir2/linkToDir3/file1"); + LRUMap cacheMap = viewfs.fsState.getPathResolutionCache(); + FileSystemTestHelper.createFile(fsTarget, resolvedPath1); + Assert.assertTrue( + resolvedPath1.toString().equals( + fileSystem.resolvePath(srcPath1).toString())); + Assert.assertEquals(viewfs.fsState.getPathResolutionCacheCapacity(), 1); + Assert.assertEquals(viewfs.fsState.getPathResolutionCache().size(), 1); + Assert.assertTrue(viewfs.fsState.getPathResolutionCache().isFull()); + InodeTree.ResolveResult resolveResult1 = + viewfs.fsState.resolve(viewfs.getUriPath(srcPath1), true); + LOG.info("Resolve result resolve path:" + resolveResult1.resolvedPath + + ", remaining path:" + resolveResult1.remainingPath + + ", file kind:" + resolveResult1.kind); + Assert.assertTrue(resolveResult1.resolvedPath.toString().equals( + new Path("/internalDir/internalDir2/linkToDir3").toString())); + Assert.assertTrue(resolveResult1.remainingPath.toString().equals( + "/file1")); + Assert.assertEquals(resolveResult1, + cacheMap.get( + InodeTree.getResolveCacheKeyStr( + viewfs.getUriPath(srcPath1), true))); + + Path srcPath2 = new Path("/internalDir/internalDir2/linkToDir3/file2"); + InodeTree.ResolveResult resolveResult2 = + viewfs.fsState.resolve(viewfs.getUriPath(srcPath2), true); + Assert.assertTrue(resolveResult2.resolvedPath.toString().equals( + new Path("/internalDir/internalDir2/linkToDir3").toString())); + Assert.assertTrue(resolveResult2.remainingPath.toString().equals( + "/file2")); + Assert.assertEquals(viewfs.fsState.getPathResolutionCache().size(), 1); + Assert.assertTrue(viewfs.fsState.getPathResolutionCache().isFull()); + Assert.assertEquals(resolveResult2, + cacheMap.get( + InodeTree.getResolveCacheKeyStr( + viewfs.getUriPath(srcPath2), true))); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md index 52ad49c57024f..62409ef8b1c24 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md @@ -366,6 +366,82 @@ Don't want to change scheme or difficult to copy mount-table configurations to a Please refer to the [View File System Overload Scheme Guide](./ViewFsOverloadScheme.html) +Regex Pattern Based Mount Points +-------------------------------- + +The view file system mount points were a Key-Value based mapping system. It is not friendly for user cases which mapping config could be abstracted to rules. E.g. Users want to provide a GCS bucket per user and there might be thousands of users in total. The old key-value based approach won't work well for several reasons: + +1. The mount table is used by FileSystem clients. There's a cost to spread the config to all clients and we should avoid it if possible. The [View File System Overload Scheme Guide](./ViewFsOverloadScheme.html) could help the distribution by central mount table management. But the mount table still have to be updated on every change. The change could be greatly avoided if provide a rule-based mount table.. + +2. The client have to understand all the KVs in the mount table. This is not ideal when the mountable grows to thousands of items. E.g. thousands of file systems might be initialized even users only need one. And the config itself will become bloated at scale. + +### Understand the Difference + +In the key-value based mount table, view file system treats every mount point as a partition. There's several file system APIs which will lead to operation on all partitions. E.g. there's an HDFS cluster with multiple mount. Users want to run “hadoop fs -put file viewfs://hdfs.namenode.apache.org/tmp/” cmd to copy data from local disk to our HDFS cluster. The cmd will trigger ViewFileSystem to call setVerifyChecksum() method which will initialize the file system for every mount point. +For a regex-base rule mount table entry, we couldn't know what's corresponding path until parsing. So the regex based mount table entry will be ignored on such cases and the file system will be created upon accessing. The inner cache of ViewFs is also not available for regex-base mount points now as it assumes target file system doesn't change after viewfs initialization. Please disable it if you want to use regex-base mount table. We also need to change the rename strategy to SAME_FILESYSTEM_ACROSS_MOUNTPOINT for the same reason. +```xml + + fs.viewfs.enable.inner.cache + false + + + fs.viewfs.rename.strategy + SAME_FILESYSTEM_ACROSS_MOUNTPOINT + +``` + +### Basic Regex Link Mapping Config +Here's an example of base regex mount point config. ${username} is the named capture group in Java Regex. +```xml + + fs.viewfs.mounttable.hadoop-nn.linkRegx./^(?\\w+) + gs://${username}.hadoop.apache.org/ + +``` +Parsing example. +```bash +viewfs://hadoop-nn/user1/dir1 => gs://user1.hadoop.apache.org/dir1 +viewfs://hadoop-nn/user2 => gs://user2.hadoop.apache.org/ +``` +The src/key’s format are +```bash +fs.viewfs.mounttable.${VIEWNAME}.linkRegx.${REGEX_STR} +``` + +### Regex Link Mapping With Interceptors +Interceptor is one mechanism introduced to modify source or target in the resolution process. It’s optional and could be used to satisfy user cases such as replace specific character or replace some word. Interceptor will only work for regex mount point. RegexMountPointResolvedDstPathReplaceInterceptor is the only build-in interceptor now. + +Here’s an example regex mount point entry with RegexMountPointResolvedDstPathReplaceInterceptor set. + +```xml + + fs.viewfs.mounttable.hadoop-nn.linkRegx.replaceresolveddstpath:_:-#./^(?\\w+) + gs://${username}.hadoop.apache.org/ + +``` +The ```replaceresolveddstpath:_:-``` is an interceptor setting. “replaceresolveddstpath” is the interceptor type, “_” is the string to replace and “-” is the string after replace. + +Parsing example. +```bash +viewfs://hadoop-nn/user_ad/dir1 => gs://user-ad.hadoop.apache.org/dir1 +viewfs://hadoop-nn/user_ad_click => gs://user-ad-click.hadoop.apache.org/ +``` +The src/key’s format are +```bash +fs.viewfs.mounttable.${VIEWNAME}.linkRegx.${REGEX_STR} +fs.viewfs.mounttable.${VIEWNAME}.linkRegx.${interceptorSettings}#.${srcRegex} +``` + +### Path Resolving Cache +Regex parsing time could vary based on the defined rule. It’s possible that the parsing time could be long. So a resolving result cache should definitely help to reduce the parsing time. Besides, the cache could also help reduce the target file system initialization time. An LRU cache is used to implement it. The cache is disabled by default. Please config the following properties to use it. +```xml + + fs.viewfs.path.resolution.cache.capacity + ${cache_cap} + +``` + + Appendix: A Mount Table Configuration Example --------------------------------------------- diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java new file mode 100644 index 0000000000000..c7731f7065f96 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -0,0 +1,320 @@ +/** + * 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.viewfs; + +import java.io.File; +import java.io.IOException; +import java.net.URI; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.FileSystemTestHelper; +import org.apache.hadoop.fs.FsConstants; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.MiniDFSNNTopology; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE; +import static org.apache.hadoop.fs.viewfs.RegexMountPoint.INTERCEPTOR_INTERNAL_SEP; + +/** + * Test linkRegex node type for view file system. + */ +public class TestViewFileSystemLinkRegex extends ViewFileSystemBaseTest { + public static final Logger LOGGER = + LoggerFactory.getLogger(TestViewFileSystemLinkRegex.class); + + private static FileSystem fsDefault; + private static MiniDFSCluster cluster; + private static Configuration clusterConfig; + private static final int NAME_SPACES_COUNT = 3; + private static final int DATA_NODES_COUNT = 3; + private static final int FS_INDEX_DEFAULT = 0; + private static final FileSystem[] FS_HDFS = new FileSystem[NAME_SPACES_COUNT]; + private static final String CLUSTER_NAME = + "TestViewFileSystemLinkRegexCluster"; + private static final File TEST_DIR = GenericTestUtils + .getTestDir(TestViewFileSystemLinkRegex.class.getSimpleName()); + private static final String TEST_BASE_PATH = + "/tmp/TestViewFileSystemLinkRegex"; + + @Override + protected FileSystemTestHelper createFileSystemHelper() { + return new FileSystemTestHelper(TEST_BASE_PATH); + } + + @BeforeClass + public static void clusterSetupAtBeginning() throws IOException { + SupportsBlocks = true; + clusterConfig = ViewFileSystemTestSetup.createConfig(); + clusterConfig.setBoolean( + DFSConfigKeys.DFS_NAMENODE_DELEGATION_TOKEN_ALWAYS_USE_KEY, + true); + cluster = new MiniDFSCluster.Builder(clusterConfig).nnTopology( + MiniDFSNNTopology.simpleFederatedTopology(NAME_SPACES_COUNT)) + .numDataNodes(DATA_NODES_COUNT).build(); + cluster.waitClusterUp(); + + for (int i = 0; i < NAME_SPACES_COUNT; i++) { + FS_HDFS[i] = cluster.getFileSystem(i); + } + fsDefault = FS_HDFS[FS_INDEX_DEFAULT]; + } + + @AfterClass + public static void clusterShutdownAtEnd() throws Exception { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Override + @Before + public void setUp() throws Exception { + fsTarget = fsDefault; + super.setUp(); + } + + /** + * Override this so that we don't set the targetTestRoot to any path under the + * root of the FS, and so that we don't try to delete the test dir, but rather + * only its contents. + */ + @Override + void initializeTargetTestRoot() throws IOException { + targetTestRoot = fsDefault.makeQualified(new Path("/")); + for (FileStatus status : fsDefault.listStatus(targetTestRoot)) { + fsDefault.delete(status.getPath(), true); + } + } + + @Override + void setupMountPoints() { + super.setupMountPoints(); + } + + @Override + int getExpectedDelegationTokenCount() { + return 1; // all point to the same fs so 1 unique token + } + + @Override + int getExpectedDelegationTokenCountWithCredentials() { + return 1; + } + + public String buildReplaceInterceptorSettingString(String srcRegex, + String replaceString) { + return + RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH.getConfigName() + + INTERCEPTOR_INTERNAL_SEP + srcRegex + INTERCEPTOR_INTERNAL_SEP + + replaceString; + } + + public String linkInterceptorSettings( + List interceptorSettingStrList) { + StringBuilder stringBuilder = new StringBuilder(); + int listSize = interceptorSettingStrList.size(); + for (int i = 0; i < listSize; ++i) { + stringBuilder.append(interceptorSettingStrList.get(i)); + if (i < listSize - 1) { + stringBuilder.append(RegexMountPoint.INTERCEPTOR_SEP); + } + } + return stringBuilder.toString(); + } + + @Test + public void testConfLinkRegexIndexMapping() throws Exception { + conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); + // (^/(\w+),/targetTestRoot/$1) + // => /targetTestRoot/testConfLinkRegexIndexMapping1 + URI viewFsUri = + new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + String regexStr = "^/(\\w+)"; + String dstPathStr = targetTestRoot + "$1"; + Path srcPath = new Path("/testConfLinkRegexIndexMapping1"); + Path expectedResolveResult = + new Path(dstPathStr.replace("$1", "testConfLinkRegexIndexMapping1")); + FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); + FileSystem vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + + // Test ${1} format + // ^/(\w+, /targetTestRoot/${1}) + // => /targetTestRoot/testConfLinkRegexIndexMapping2 + dstPathStr = targetTestRoot + "${1}"; + srcPath = new Path("/testConfLinkRegexIndexMapping2"); + expectedResolveResult = + new Path(dstPathStr.replace("${1}", "testConfLinkRegexIndexMapping2")); + outputStream = fsTarget.create(expectedResolveResult); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); + vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + + //(^/(\w+)/file1, /targetTestRoot/$1) + // = > /targetTestRoot/testConfLinkRegexIndexMapping3/file1 + dstPathStr = targetTestRoot + "$1"; + srcPath = new Path("/testConfLinkRegexIndexMapping3/file1"); + expectedResolveResult = new Path( + dstPathStr.replace("$1", "testConfLinkRegexIndexMapping3/file1")); + outputStream = fsTarget.create(expectedResolveResult); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); + vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + + //(^/(\w+)/file1, /targetTestRoot/$1/) + // = > /targetTestRoot/testConfLinkRegexIndexMapping4/file1 + dstPathStr = targetTestRoot + "$1/"; + srcPath = new Path("/testConfLinkRegexIndexMapping4/file1"); + expectedResolveResult = new Path( + dstPathStr.replace("$1", "testConfLinkRegexIndexMapping4/file1")); + outputStream = fsTarget.create(expectedResolveResult); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); + vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + } + + @Test + public void testConfLinkRegexNamedGroupMapping() throws Exception { + conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); + // ^/(?\\w+) = > /targetTestRoot/$firstDir + URI viewFsUri = + new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + String regexStr = "^/(?\\w+)"; + String dstPathStr = targetTestRoot + "$firstDir"; + Path srcPath = new Path("/testConfLinkRegexNamedGroupMapping1"); + Path expectedResolveResult = new Path( + dstPathStr.replace("$firstDir", "testConfLinkRegexNamedGroupMapping1")); + FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); + FileSystem vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + + // Test ${1} format + dstPathStr = targetTestRoot + "${firstDir}"; + srcPath = new Path("/testConfLinkRegexNamedGroupMapping2"); + expectedResolveResult = new Path(dstPathStr + .replace("${firstDir}", "testConfLinkRegexNamedGroupMapping2")); + outputStream = fsTarget.create(expectedResolveResult); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); + vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + } + + @Test + public void testConfLinkRegexFixedDestMapping() throws Exception { + conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); + URI viewFsUri = + new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + String regexStr = "^/\\w+"; + String dstPathStr = + targetTestRoot + "testConfLinkRegexFixedDestMappingFile"; + Path expectedResolveResult = new Path(dstPathStr); + FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); + FileSystem vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue( + expectedResolveResult.equals(vfs.resolvePath(new Path("/misc1")))); + Assert.assertTrue( + expectedResolveResult.equals(vfs.resolvePath(new Path("/misc2")))); + } + + @Test + public void testConfLinkRegexWithSingleInterceptor() throws Exception { + conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); + URI viewFsUri = + new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + String regexStr = "^/user/(?\\w+)"; + String dstPathStr = targetTestRoot + "$username"; + // Replace "_" with "-" + String settingString = buildReplaceInterceptorSettingString("_", "-"); + Path srcPath = new Path("/user/hadoop_user1/hadoop_file1"); + Path expectedResolveResult = + new Path(targetTestRoot, "hadoop-user1/hadoop_file1"); + FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil + .addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, settingString); + FileSystem vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + } + + @Test + public void testConfLinkRegexWithInterceptors() throws Exception { + conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); + URI viewFsUri = + new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + String regexStr = "^/user/(?\\w+)/"; + String dstPathStr = targetTestRoot + "$username"; + // Replace "_" with "-" + String interceptor1 = buildReplaceInterceptorSettingString("_", "-"); + // Replace "hadoop" with "hdfs" + String interceptor2 = + buildReplaceInterceptorSettingString("hadoop", "hdfs"); + String interceptors = + linkInterceptorSettings(Arrays.asList(interceptor1, interceptor2)); + Path srcPath = new Path("/user/hadoop_user1/hadoop_file1"); + Path expectedResolveResult = + new Path(targetTestRoot, "hdfs-user1/hadoop_file1"); + FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); + fsTarget.listStatus(expectedResolveResult); + outputStream.close(); + ConfigUtil + .addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, interceptors); + FileSystem vfs = FileSystem.get(viewFsUri, conf); + Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); + Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + } +} From d66b86b5e5d9fd5b7e8a40f43fedc93c33f44a24 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Mon, 17 Aug 2020 20:26:14 -0700 Subject: [PATCH 02/12] HADOOP-15891. address comments. --- .../apache/hadoop/fs/viewfs/ConfigUtil.java | 19 +- .../apache/hadoop/fs/viewfs/Constants.java | 1 + .../apache/hadoop/fs/viewfs/InodeTree.java | 19 +- .../hadoop/fs/viewfs/RegexMountPoint.java | 140 +++++-- .../hadoop/fs/viewfs/ViewFileSystem.java | 21 +- .../org/apache/hadoop/fs/viewfs/ViewFs.java | 8 + .../hadoop/fs/viewfs/TestRegexMountPoint.java | 18 +- ...TestRegexMountPointInterceptorFactory.java | 6 +- ...ointResolvedDstPathReplaceInterceptor.java | 15 +- .../hadoop/fs/viewfs/TestViewFsConfig.java | 6 + .../fs/viewfs/ViewFileSystemBaseTest.java | 4 +- .../hadoop-hdfs/src/site/markdown/ViewFs.md | 8 +- .../viewfs/TestViewFileSystemLinkRegex.java | 369 ++++++++++++------ 13 files changed, 443 insertions(+), 191 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java index fdb0a2c204cac..91f87448cd342 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java @@ -169,10 +169,10 @@ public static void addLinkNfly(final Configuration conf, final String src, /** * Add a LinkRegex to the config for the specified mount table. - * @param conf - * @param mountTableName - * @param srcRegex - * @param targetStr + * @param conf - get mountable config from this conf + * @param mountTableName - the mountable name of the regex config item + * @param srcRegex - the src path regex expression that applies to this config + * @param targetStr - the string of target path */ public static void addLinkRegex( Configuration conf, final String mountTableName, @@ -182,11 +182,12 @@ public static void addLinkRegex( /** * Add a LinkRegex to the config for the specified mount table. - * @param conf - * @param mountTableName - * @param srcRegex - * @param targetStr - * @param interceptorSettings + * @param conf - get mountable config from this conf + * @param mountTableName - the mountable name of the regex config item + * @param srcRegex - the src path regex expression that applies to this config + * @param targetStr - the string of target path + * @param interceptorSettings - the serialized interceptor string to be + * applied while resolving the mapping */ public static void addLinkRegex( Configuration conf, final String mountTableName, final String srcRegex, diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index 53c55eef0e8e2..947ec3f39ee17 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -100,6 +100,7 @@ public interface Constants { /** * Enable ViewFileSystem to cache all children filesystems in inner cache. + * It won't work for regex based mount points now. */ String CONFIG_VIEWFS_ENABLE_INNER_CACHE = "fs.viewfs.enable.inner.cache"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 70adb32e38631..6f835b4dfc97e 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -243,7 +243,7 @@ enum LinkType { /** * Link entry which source are regex exrepssions and target refer matched * group from source - * Config prefix: fs.viewfs.mounttable..linkMerge + * Config prefix: fs.viewfs.mounttable..linkRegex * Refer: {@link Constants#CONFIG_VIEWFS_LINK_REGEX} */ REGEX; @@ -394,6 +394,9 @@ private void createLink(final String src, final String target, protected abstract T getTargetFileSystem(URI uri) throws UnsupportedFileSystemException, URISyntaxException, IOException; + protected abstract T getTargetFileSystem(URI uri, boolean enableCache) + throws UnsupportedFileSystemException, URISyntaxException, IOException; + protected abstract T getTargetFileSystem(INodeDir dir) throws URISyntaxException, IOException; @@ -844,6 +847,10 @@ ResolveResult resolve(final String p, final boolean resolveLastComponent) /** * Walk through all regex mount points to see * whether the path match any regex expressions. + * E.g. link: ^/user/(?\\w+) => s3://$user.apache.com/_${user} + * srcPath: is /user/hadoop/dir1 + * resolveLastComponent: true + * then return value is s3://hadoop.apache.com/_hadoop * * @param srcPath * @param resolveLastComponent @@ -874,7 +881,8 @@ protected ResolveResult buildResolveResultForRegexMountPoint( ResultKind resultKind, String resolvedPathStr, String targetOfResolvedPathStr, Path remainingPath) { try { - T targetFs = getTargetFileSystem(new URI(targetOfResolvedPathStr)); + T targetFs = getTargetFileSystem( + new URI(targetOfResolvedPathStr), false); return new ResolveResult(resultKind, targetFs, resolvedPathStr, remainingPath); } catch (IOException ex) { @@ -911,8 +919,8 @@ private void addResolveResultToCache(final String pathStr, final Boolean resolveLastComponent, final ResolveResult resolveResult) { try { - cacheRWLock.writeLock().lock(); String key = getResolveCacheKeyStr(pathStr, resolveLastComponent); + cacheRWLock.writeLock().lock(); pathResolutionCache.put(key, resolveResult); } finally { cacheRWLock.writeLock().unlock(); @@ -925,8 +933,8 @@ private ResolveResult getResolveResultFromCache(final String pathStr, return null; } try { - cacheRWLock.readLock().lock(); String key = getResolveCacheKeyStr(pathStr, resolveLastComponent); + cacheRWLock.readLock().lock(); return (ResolveResult) pathResolutionCache.get(key); } finally { cacheRWLock.readLock().unlock(); @@ -938,7 +946,8 @@ public static String getResolveCacheKeyStr(final String path, return path + ",resolveLastComp" + resolveLastComp; } - @VisibleForTesting public LRUMap getPathResolutionCache() { + @VisibleForTesting + public LRUMap getPathResolutionCache() { return pathResolutionCache; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java index 5d8049b049658..e5d6c16d5aef1 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java @@ -60,6 +60,8 @@ class RegexMountPoint { public static final Pattern VAR_PATTERN_IN_DEST = Pattern.compile("\\$((\\{\\w+\\})|(\\w+))"); + // Same var might have different representations. + // e.g. // key => $key or key = > ${key} private Map> varInDestPathMap; @@ -115,7 +117,7 @@ private void initializeInterceptors() throws IOException { /** * Get $var1 and $var2 style variables in string. * - * @param input + * @param input - the string to be process. * @return */ public static Map> getVarListInString(String input) { @@ -155,24 +157,21 @@ public static Pattern getVarPatternInDest() { /** * Get resolved path from regex mount points. - * @param srcPath - * @param resolveLastComponent - * @return + * E.g. link: ^/user/(?\\w+) => s3://$user.apache.com/_${user} + * srcPath: is /user/hadoop/dir1 + * resolveLastComponent: true + * then return value is s3://hadoop.apache.com/_hadoop + * @param srcPath - the src path to resolve + * @param resolveLastComponent - whether resolve the path after last `/` + * @return mapped path of the mount point. */ public InodeTree.ResolveResult resolve(final String srcPath, final boolean resolveLastComponent) { - String pathStrToResolve = srcPath; - if (!resolveLastComponent) { - int lastSlashIndex = srcPath.lastIndexOf(SlashPath.toString()); - if (lastSlashIndex == -1) { - return null; - } - pathStrToResolve = srcPath.substring(0, lastSlashIndex); - } + String pathStrToResolve = getPathToResolve(srcPath, resolveLastComponent); for (RegexMountPointInterceptor interceptor : interceptorList) { pathStrToResolve = interceptor.interceptSource(pathStrToResolve); } - LOGGER.debug("Path to resolve:" + pathStrToResolve + ",srcPattern:" + LOGGER.debug("Path to resolve:" + pathStrToResolve + ", srcPattern:" + getSrcPathRegex()); Matcher srcMatcher = getSrcPattern().matcher(pathStrToResolve); String parsedDestPath = getDstPath(); @@ -182,37 +181,18 @@ public InodeTree.ResolveResult resolve(final String srcPath, resolvedPathStr = pathStrToResolve.substring(0, srcMatcher.end()); Map> varMap = getVarInDestPathMap(); for (Map.Entry> entry : varMap.entrySet()) { - String groupNameOrIndexStr = entry.getKey(); - String groupValue = null; - if (groupNameOrIndexStr.matches("\\d+")) { - // group index - int groupIndex = Integer.parseUnsignedInt(groupNameOrIndexStr); - if (groupIndex >= 0 && groupIndex <= srcMatcher.groupCount()) { - groupValue = srcMatcher.group(groupIndex); - } - } else { - // named group in regex - groupValue = srcMatcher.group(groupNameOrIndexStr); - } - if (groupValue == null) { - continue; - } - Set varNameListToReplace = entry.getValue(); - for (String varName : varNameListToReplace) { - parsedDestPath = parsedDestPath.replace(varName, groupValue); - LOGGER.debug("parsedDestPath value is:" + parsedDestPath); - } + String regexGroupNameOrIndexStr = entry.getKey(); + Set groupRepresentationStrSetInDest = entry.getValue(); + parsedDestPath = replaceRegexCaptureGroupInPath( + parsedDestPath, srcMatcher, + regexGroupNameOrIndexStr, groupRepresentationStrSetInDest); } ++mappedCount; } if (0 == mappedCount) { return null; } - String remainingPathStr = srcPath.substring(resolvedPathStr.length()); - if (!remainingPathStr.startsWith("/")) { - remainingPathStr = "/" + remainingPathStr; - } - Path remainingPath = new Path(remainingPathStr); + Path remainingPath = getRemainingPathStr(srcPath, resolvedPathStr); for (RegexMountPointInterceptor interceptor : interceptorList) { parsedDestPath = interceptor.interceptResolvedDestPathStr(parsedDestPath); remainingPath = @@ -224,10 +204,92 @@ public InodeTree.ResolveResult resolve(final String srcPath, return resolveResult; } + private Path getRemainingPathStr( + String srcPath, + String resolvedPathStr) { + String remainingPathStr = srcPath.substring(resolvedPathStr.length()); + if (!remainingPathStr.startsWith("/")) { + remainingPathStr = "/" + remainingPathStr; + } + return new Path(remainingPathStr); + } + + private String getPathToResolve( + String srcPath, boolean resolveLastComponent) { + if (resolveLastComponent) { + return srcPath; + } + int lastSlashIndex = srcPath.lastIndexOf(SlashPath.toString()); + if (lastSlashIndex == -1) { + return null; + } + return srcPath.substring(0, lastSlashIndex); + } + + /** + * Use capture group named regexGroupNameOrIndexStr in mather to replace + * parsedDestPath. + * E.g. link: ^/user/(?\\w+) => s3://$user.apache.com/_${user} + * srcMatcher is from /user/hadoop. + * Then the params will be like following. + * parsedDestPath: s3://$user.apache.com/_${user}, + * regexGroupNameOrIndexStr: user + * groupRepresentationStrSetInDest: {user:$user; user:${user}} + * return value will be s3://hadoop.apache.com/_hadoop + * @param parsedDestPath + * @param srcMatcher + * @param regexGroupNameOrIndexStr + * @param groupRepresentationStrSetInDest + * @return return parsedDestPath while ${var},$var replaced or + * parsedDestPath nothing found. + */ + private String replaceRegexCaptureGroupInPath( + String parsedDestPath, + Matcher srcMatcher, + String regexGroupNameOrIndexStr, + Set groupRepresentationStrSetInDest) { + String groupValue = getRegexGroupValueFromMather( + srcMatcher, regexGroupNameOrIndexStr); + if (groupValue == null) { + return parsedDestPath; + } + for (String varName : groupRepresentationStrSetInDest) { + parsedDestPath = parsedDestPath.replace(varName, groupValue); + LOGGER.debug("parsedDestPath value is:" + parsedDestPath); + } + return parsedDestPath; + } + + /** + * Get matched capture group value from regex matched string. E.g. + * Regex: ^/user/(?\\w+), regexGroupNameOrIndexStr: userName + * then /user/hadoop should return hadoop while call + * getRegexGroupValueFromMather(matcher, usersName) + * or getRegexGroupValueFromMather(matcher, 1) + * + * @param srcMatcher - the matcher to be use + * @param regexGroupNameOrIndexStr - the regex group name or index + * @return - Null if no matched group named regexGroupNameOrIndexStr found. + */ + private String getRegexGroupValueFromMather( + Matcher srcMatcher, String regexGroupNameOrIndexStr) { + if (regexGroupNameOrIndexStr.matches("\\d+")) { + // group index + int groupIndex = Integer.parseUnsignedInt(regexGroupNameOrIndexStr); + if (groupIndex >= 0 && groupIndex <= srcMatcher.groupCount()) { + return srcMatcher.group(groupIndex); + } + } else { + // named group in regex + return srcMatcher.group(regexGroupNameOrIndexStr); + } + return null; + } + /** * Convert interceptor to string. * - * @param interceptorList + * @param interceptorList - the interceptor list to be applied. * @return */ public static String convertInterceptorsToString( diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index 1ba91b5edfa1c..b173271134987 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -300,13 +300,20 @@ public void initialize(final URI theUri, final Configuration conf) @Override protected FileSystem getTargetFileSystem(final URI uri) throws URISyntaxException, IOException { - FileSystem fs; - if (enableInnerCache) { - fs = innerCache.get(uri, config); - } else { - fs = fsGetter.get(uri, config); - } - return new ChRootedFileSystem(fs, uri); + return getTargetFileSystem(uri, enableInnerCache); + } + + @Override + protected FileSystem getTargetFileSystem( + final URI uri, boolean enableCache) + throws URISyntaxException, IOException { + FileSystem fs; + if (enableCache) { + fs = innerCache.get(uri, config); + } else { + fs = fsGetter.get(uri, config); + } + return new ChRootedFileSystem(fs, uri); } @Override 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 95b596bde367d..d3d6373d27596 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 @@ -248,6 +248,14 @@ protected AbstractFileSystem getTargetFileSystem(final URI uri) new Path(pathString)); } + @Override + protected AbstractFileSystem getTargetFileSystem( + final URI uri, boolean enableCache) + throws URISyntaxException, UnsupportedFileSystemException { + throw new UnsupportedFileSystemException( + "Cache option is not supported in ViewFs! Uri:" + uri); + } + @Override protected AbstractFileSystem getTargetFileSystem( final INodeDir dir) throws URISyntaxException { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java index 760df4b84650f..34b3b8fdf4b4e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java @@ -68,6 +68,12 @@ protected TestRegexMountPointFileSystem getTargetFileSystem( return new TestRegexMountPointFileSystem(uri); } + @Override + protected TestRegexMountPointFileSystem getTargetFileSystem( + final URI uri, boolean enableCache) { + return new TestRegexMountPointFileSystem(uri); + } + @Override protected TestRegexMountPointFileSystem getTargetFileSystem( final INodeDir dir) { @@ -82,11 +88,13 @@ protected TestRegexMountPointFileSystem getTargetFileSystem( }; } - @After public void tearDown() throws Exception { + @After + public void tearDown() throws Exception { inodeTree = null; } - @Test public void testGetVarListInString() throws IOException { + @Test + public void testGetVarListInString() throws IOException { String srcRegex = "/(\\w+)"; String target = "/$0/${1}/$1/${2}/${2}"; RegexMountPoint regexMountPoint = @@ -103,7 +111,8 @@ protected TestRegexMountPointFileSystem getTargetFileSystem( Assert.assertTrue(varMap.get("2").contains("${2}")); } - @Test public void testResolve() throws IOException { + @Test + public void testResolve() throws IOException { String regexStr = "^/user/(?\\w+)"; String dstPathStr = "/namenode1/testResolve/$username"; String settingsStr = null; @@ -126,7 +135,8 @@ protected TestRegexMountPointFileSystem getTargetFileSystem( Assert.assertTrue(resolveResult.remainingPath.toString().equals("/file1")); } - @Test public void testResolveWithInterceptor() throws IOException { + @Test + public void testResolveWithInterceptor() throws IOException { String regexStr = "^/user/(?\\w+)"; String dstPathStr = "/namenode1/testResolve/$username"; // Replace "_" with "-" diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java index 0b12332353067..c567944ffe307 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java @@ -25,7 +25,8 @@ */ public class TestRegexMountPointInterceptorFactory { - @Test public void testCreateNormalCase() { + @Test + public void testCreateNormalCase() { String replaceInterceptorStr = RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH.getConfigName() + Character.toString(RegexMountPoint.INTERCEPTOR_INTERNAL_SEP) @@ -38,7 +39,8 @@ public class TestRegexMountPointInterceptorFactory { instanceof RegexMountPointResolvedDstPathReplaceInterceptor); } - @Test public void testCreateBadCase() { + @Test + public void testCreateBadCase() { String replaceInterceptorStr = RegexMountPointInterceptorType.REPLACE_RESOLVED_DST_PATH.getConfigName() + "___" + Character diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java index 5c44fbb25c089..598d5c759d95d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java @@ -36,7 +36,8 @@ public String createSerializedString(String regex, String replaceString) { + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + replaceString; } - @Test public void testDeserializeFromStringNormalCase() throws IOException { + @Test + public void testDeserializeFromStringNormalCase() throws IOException { String srcRegex = "-"; String replaceString = "_"; String serializedString = createSerializedString(srcRegex, replaceString); @@ -51,7 +52,8 @@ public String createSerializedString(String regex, String replaceString) { interceptor.getSrcRegexPattern().toString().equals(srcRegex)); } - @Test public void testDeserializeFromStringBadCase() throws IOException { + @Test + public void testDeserializeFromStringBadCase() throws IOException { String srcRegex = "-"; String replaceString = "_"; String serializedString = createSerializedString(srcRegex, replaceString); @@ -62,7 +64,8 @@ public String createSerializedString(String regex, String replaceString) { Assert.assertEquals(interceptor, null); } - @Test public void testSerialization() { + @Test + public void testSerialization() { String srcRegex = "word1"; String replaceString = "word2"; String serializedString = createSerializedString(srcRegex, replaceString); @@ -72,7 +75,8 @@ public String createSerializedString(String regex, String replaceString) { Assert.assertEquals(interceptor.serializeToString(), serializedString); } - @Test public void testInterceptSource() { + @Test + public void testInterceptSource() { String srcRegex = "word1"; String replaceString = "word2"; RegexMountPointResolvedDstPathReplaceInterceptor interceptor = @@ -82,7 +86,8 @@ public String createSerializedString(String regex, String replaceString) { sourcePath = interceptor.interceptSource(sourcePath); } - @Test public void testInterceptResolve() throws IOException { + @Test + public void testInterceptResolve() throws IOException { String pathAfterResolution = "/user-hadoop"; Path remainingPath = new Path("/ad-data"); diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java index 56f5b2d997dc2..ec1a97bf3b86d 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java @@ -46,6 +46,12 @@ protected Foo getTargetFileSystem(final URI uri) { return null; } + @Override + protected Foo getTargetFileSystem( + final URI uri, boolean enableCache) { + return null; + } + @Override protected Foo getTargetFileSystem(final INodeDir dir) { return null; diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index e6b4c53543b83..26895926037e1 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -1418,8 +1418,7 @@ public void testGetContentSummaryWithFileInLocalFS() throws Exception { fos.write(expected.getBytes()); } ConfigUtil.addLink(conf, - "/internalDir/internalDir2/linkToLocalFile", - localFile.toURI()); + "/internalDir/internalDir2/linkToLocalFile", localFile.toURI()); try (FileSystem fs = FileSystem.get(FsConstants.VIEWFS_URI, conf)) { ContentSummary summaryAfter = @@ -1436,7 +1435,6 @@ public void testGetContentSummaryWithFileInLocalFS() throws Exception { @Test public void testMountPointCache() throws Exception { conf.setInt(Constants.CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY, 1); - conf.setBoolean("fs.viewfs.impl.disable.cache", true); FileSystem fileSystem = FileSystem.get(FsConstants.VIEWFS_URI, conf); ViewFileSystem viewfs = (ViewFileSystem) fileSystem; Path resolvedPath1 = new Path(targetTestRoot, "dir3/file1"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md index 62409ef8b1c24..571cb49200220 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md @@ -371,19 +371,15 @@ Regex Pattern Based Mount Points The view file system mount points were a Key-Value based mapping system. It is not friendly for user cases which mapping config could be abstracted to rules. E.g. Users want to provide a GCS bucket per user and there might be thousands of users in total. The old key-value based approach won't work well for several reasons: -1. The mount table is used by FileSystem clients. There's a cost to spread the config to all clients and we should avoid it if possible. The [View File System Overload Scheme Guide](./ViewFsOverloadScheme.html) could help the distribution by central mount table management. But the mount table still have to be updated on every change. The change could be greatly avoided if provide a rule-based mount table.. +1. The mount table is used by FileSystem clients. There's a cost to spread the config to all clients and we should avoid it if possible. The [View File System Overload Scheme Guide](./ViewFsOverloadScheme.html) could help the distribution by central mount table management. But the mount table still have to be updated on every change. The change could be greatly avoided if provide a rule-based mount table. 2. The client have to understand all the KVs in the mount table. This is not ideal when the mountable grows to thousands of items. E.g. thousands of file systems might be initialized even users only need one. And the config itself will become bloated at scale. ### Understand the Difference In the key-value based mount table, view file system treats every mount point as a partition. There's several file system APIs which will lead to operation on all partitions. E.g. there's an HDFS cluster with multiple mount. Users want to run “hadoop fs -put file viewfs://hdfs.namenode.apache.org/tmp/” cmd to copy data from local disk to our HDFS cluster. The cmd will trigger ViewFileSystem to call setVerifyChecksum() method which will initialize the file system for every mount point. -For a regex-base rule mount table entry, we couldn't know what's corresponding path until parsing. So the regex based mount table entry will be ignored on such cases and the file system will be created upon accessing. The inner cache of ViewFs is also not available for regex-base mount points now as it assumes target file system doesn't change after viewfs initialization. Please disable it if you want to use regex-base mount table. We also need to change the rename strategy to SAME_FILESYSTEM_ACROSS_MOUNTPOINT for the same reason. +For a regex-base rule mount table entry, we couldn't know what's corresponding path until parsing. So the regex based mount table entry will be ignored on such cases. The file system will be created upon accessing. The inner cache of ViewFs is also not available for regex-base mount points now as it assumes target file system doesn't change after viewfs initialization. ```xml - - fs.viewfs.enable.inner.cache - false - fs.viewfs.rename.strategy SAME_FILESYSTEM_ACROSS_MOUNTPOINT diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java index c7731f7065f96..c341e8fb8b111 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -20,6 +20,8 @@ import java.io.File; import java.io.IOException; import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -42,7 +44,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE; import static org.apache.hadoop.fs.viewfs.RegexMountPoint.INTERCEPTOR_INTERNAL_SEP; /** @@ -152,148 +153,297 @@ public String linkInterceptorSettings( return stringBuilder.toString(); } + private void createDirWithChildren( + FileSystem fileSystem, Path dir, List childrenFiles) + throws IOException { + Assert.assertTrue(fileSystem.mkdirs(dir)); + int index = 0; + for (Path childFile : childrenFiles) { + createFile(fileSystem, childFile, index, true); + } + } + + private void createFile( + FileSystem fileSystem, Path file, int dataLenToWrite, boolean overwrite) + throws IOException { + FSDataOutputStream outputStream = null; + try { + outputStream = fileSystem.create(file, overwrite); + for (int i = 0; i < dataLenToWrite; ++i) { + outputStream.writeByte(i); + } + outputStream.close(); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + } + + private void createDirWithChildren( + FileSystem fileSystem, Path dir, int childrenFilesCnt) + throws IOException { + List childrenFiles = new ArrayList<>(childrenFilesCnt); + for (int i = 0; i < childrenFilesCnt; ++i) { + childrenFiles.add(new Path(dir, "file" + i)); + } + createDirWithChildren(fileSystem, dir, childrenFiles); + } + + /** + * The function used to test regex mountpoints. + * @param config - get mountable config from this conf + * @param regexStr - the src path regex expression that applies to this config + * @param dstPathStr - the string of target path + * @param interceptorSettings - the serialized interceptor string to be + * applied while resolving the mapping + * @param dirPathBeforeMountPoint - the src path user passed in to be mapped. + * @param expectedResolveResult - the expected path after resolve + * dirPathBeforeMountPoint via regex mountpint. + * @param childrenFilesCnt - the child files under dirPathBeforeMountPoint to + * be created + * @throws IOException + * @throws URISyntaxException + */ + private void testRegexMountpoint( + Configuration config, + String regexStr, + String dstPathStr, + String interceptorSettings, + Path dirPathBeforeMountPoint, + Path expectedResolveResult, + int childrenFilesCnt) + throws IOException, URISyntaxException { + FileSystem vfs = null; + try { + // Set up test env + createDirWithChildren( + fsTarget, expectedResolveResult, childrenFilesCnt); + ConfigUtil.addLinkRegex( + config, CLUSTER_NAME, regexStr, dstPathStr, interceptorSettings); + + // Asserts + URI viewFsUri = new URI( + FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + vfs = FileSystem.get(viewFsUri, config); + Assert.assertTrue( + expectedResolveResult.equals( + vfs.resolvePath(dirPathBeforeMountPoint))); + Assert.assertTrue( + vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory()); + Assert.assertEquals( + childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); + } finally { + if (vfs != null) { + vfs.close(); + } + } + } + /** + * Test regex mount points which use capture group index for mapping. + * + * @throws Exception + */ @Test public void testConfLinkRegexIndexMapping() throws Exception { - conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); - // (^/(\w+),/targetTestRoot/$1) - // => /targetTestRoot/testConfLinkRegexIndexMapping1 - URI viewFsUri = - new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + // CHECKSTYLE:OFF + // Config: + // + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // /targetTestRoot/$1 + // + // Dir path to test: /testConfLinkRegexIndexMapping1 + // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping1 + // CHECKSTYLE:ON String regexStr = "^/(\\w+)"; String dstPathStr = targetTestRoot + "$1"; Path srcPath = new Path("/testConfLinkRegexIndexMapping1"); - Path expectedResolveResult = - new Path(dstPathStr.replace("$1", "testConfLinkRegexIndexMapping1")); - FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); - FileSystem vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); - - // Test ${1} format - // ^/(\w+, /targetTestRoot/${1}) - // => /targetTestRoot/testConfLinkRegexIndexMapping2 + Path expectedResolveResult = new Path(dstPathStr.replace( + "$1", "testConfLinkRegexIndexMapping1")); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + srcPath, expectedResolveResult, 3); + + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // /targetTestRoot/${1} + // + // Dir path to test: /testConfLinkRegexIndexMapping2 + // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping2 + + // CHECKSTYLE:ON dstPathStr = targetTestRoot + "${1}"; srcPath = new Path("/testConfLinkRegexIndexMapping2"); expectedResolveResult = - new Path(dstPathStr.replace("${1}", "testConfLinkRegexIndexMapping2")); - outputStream = fsTarget.create(expectedResolveResult); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); - vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); - - //(^/(\w+)/file1, /targetTestRoot/$1) - // = > /targetTestRoot/testConfLinkRegexIndexMapping3/file1 + new Path( + dstPathStr.replace("${1}", "testConfLinkRegexIndexMapping2")); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + srcPath, expectedResolveResult, 4); + + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // /targetTestRoot/$1 + // + // Dir path to test: /testConfLinkRegexIndexMapping3/dir1 + // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping3/dir1 + // CHECKSTYLE:ON dstPathStr = targetTestRoot + "$1"; - srcPath = new Path("/testConfLinkRegexIndexMapping3/file1"); + srcPath = new Path("/testConfLinkRegexIndexMapping3/dir1"); expectedResolveResult = new Path( - dstPathStr.replace("$1", "testConfLinkRegexIndexMapping3/file1")); - outputStream = fsTarget.create(expectedResolveResult); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); - vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); - - //(^/(\w+)/file1, /targetTestRoot/$1/) - // = > /targetTestRoot/testConfLinkRegexIndexMapping4/file1 - dstPathStr = targetTestRoot + "$1/"; - srcPath = new Path("/testConfLinkRegexIndexMapping4/file1"); + dstPathStr.replace("$1", "testConfLinkRegexIndexMapping3/dir1")); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + srcPath, expectedResolveResult, 5); + + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // /targetTestRoot/${1}/ + // + // Dir path to test: /testConfLinkRegexIndexMapping4/dir1 + // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping4/dir1 + // CHECKSTYLE:ON + dstPathStr = targetTestRoot + "${1}/"; + srcPath = new Path("/testConfLinkRegexIndexMapping4/dir1"); expectedResolveResult = new Path( - dstPathStr.replace("$1", "testConfLinkRegexIndexMapping4/file1")); - outputStream = fsTarget.create(expectedResolveResult); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); - vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + dstPathStr.replace("${1}", "testConfLinkRegexIndexMapping4/dir1")); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + srcPath, expectedResolveResult, 6); } + /** + * Test regex mount pointes with named capture group. + * @throws Exception + */ @Test public void testConfLinkRegexNamedGroupMapping() throws Exception { - conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); - // ^/(?\\w+) = > /targetTestRoot/$firstDir - URI viewFsUri = - new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(?\w+) + // /targetTestRoot/$firstDir + // + // Dir path to test: /testConfLinkRegexNamedGroupMapping1 + // Expect path: /targetTestRoot/testConfLinkRegexNamedGroupMapping1 + // CHECKSTYLE:ON + URI viewFsUri = new URI( + FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); String regexStr = "^/(?\\w+)"; String dstPathStr = targetTestRoot + "$firstDir"; Path srcPath = new Path("/testConfLinkRegexNamedGroupMapping1"); Path expectedResolveResult = new Path( dstPathStr.replace("$firstDir", "testConfLinkRegexNamedGroupMapping1")); - FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); - FileSystem vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); - - // Test ${1} format + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + srcPath, expectedResolveResult, 3); + + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(?\w+) + // /targetTestRoot/${firstDir} + // + // Dir path to test: /testConfLinkRegexNamedGroupMapping2 + // Expect path: /targetTestRoot/testConfLinkRegexNamedGroupMapping2 + // CHECKSTYLE:ON dstPathStr = targetTestRoot + "${firstDir}"; srcPath = new Path("/testConfLinkRegexNamedGroupMapping2"); - expectedResolveResult = new Path(dstPathStr - .replace("${firstDir}", "testConfLinkRegexNamedGroupMapping2")); - outputStream = fsTarget.create(expectedResolveResult); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); - vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + expectedResolveResult = new Path( + dstPathStr.replace("${firstDir}", "testConfLinkRegexNamedGroupMapping2")); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + srcPath, expectedResolveResult, 5); } + /** + * Test cases when the destination is fixed paths. + * @throws Exception + */ @Test public void testConfLinkRegexFixedDestMapping() throws Exception { - conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); - URI viewFsUri = - new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(?\w+) + // /targetTestRoot/${firstDir} + // + // Dir path to test: /misc1 + // Expect path: /targetTestRoot/testConfLinkRegexFixedDestMappingFile + // Dir path to test: /misc2 + // Expect path: /targetTestRoot/testConfLinkRegexFixedDestMappingFile + // CHECKSTYLE:ON String regexStr = "^/\\w+"; String dstPathStr = targetTestRoot + "testConfLinkRegexFixedDestMappingFile"; Path expectedResolveResult = new Path(dstPathStr); - FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil.addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, null); - FileSystem vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue( - expectedResolveResult.equals(vfs.resolvePath(new Path("/misc1")))); - Assert.assertTrue( - expectedResolveResult.equals(vfs.resolvePath(new Path("/misc2")))); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + new Path("/misc1"), expectedResolveResult, 5); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, null, + new Path("/misc2"), expectedResolveResult, 6); } + /** + * Test regex mount point config with a single interceptor. + * + */ @Test public void testConfLinkRegexWithSingleInterceptor() throws Exception { - conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); - URI viewFsUri = - new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.replaceresolveddstpath:_:-#.^/user/(?\w+) + // /targetTestRoot/$username + // + // Dir path to test: /user/hadoop_user1/hadoop_dir1 + // Expect path: /targetTestRoot/hadoop-user1/hadoop_dir1 + // CHECKSTYLE:ON + String regexStr = "^/user/(?\\w+)"; String dstPathStr = targetTestRoot + "$username"; // Replace "_" with "-" String settingString = buildReplaceInterceptorSettingString("_", "-"); - Path srcPath = new Path("/user/hadoop_user1/hadoop_file1"); - Path expectedResolveResult = - new Path(targetTestRoot, "hadoop-user1/hadoop_file1"); - FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil - .addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, settingString); - FileSystem vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + Path srcPath = new Path("/user/hadoop_user1/hadoop_dir1"); + Path expectedResolveResult = new Path( + targetTestRoot, "hadoop-user1/hadoop_dir1"); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, settingString, + srcPath, expectedResolveResult, 2); } + /** + * Test regex mount point config with multiple interceptors. + * + */ @Test public void testConfLinkRegexWithInterceptors() throws Exception { - conf.setBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, false); + // CHECKSTYLE:OFF + // Config: + // + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.replaceresolveddstpath:_:-;replaceresolveddstpath:hadoop:hdfs#.^/user/(?\w+) + // /targetTestRoot/$username + // + // Dir path to test: /user/hadoop_user1/hadoop_dir1 + // Expect path: /targetTestRoot/hdfs-user1/hadoop_dir1 + // CHECKSTYLE:ON URI viewFsUri = new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); String regexStr = "^/user/(?\\w+)/"; @@ -305,16 +455,13 @@ public void testConfLinkRegexWithInterceptors() throws Exception { buildReplaceInterceptorSettingString("hadoop", "hdfs"); String interceptors = linkInterceptorSettings(Arrays.asList(interceptor1, interceptor2)); - Path srcPath = new Path("/user/hadoop_user1/hadoop_file1"); + Path srcPath = new Path("/user/hadoop_user1/hadoop_dir1"); Path expectedResolveResult = - new Path(targetTestRoot, "hdfs-user1/hadoop_file1"); - FSDataOutputStream outputStream = fsTarget.create((expectedResolveResult)); - fsTarget.listStatus(expectedResolveResult); - outputStream.close(); - ConfigUtil - .addLinkRegex(conf, CLUSTER_NAME, regexStr, dstPathStr, interceptors); - FileSystem vfs = FileSystem.get(viewFsUri, conf); - Assert.assertTrue(expectedResolveResult.equals(vfs.resolvePath(srcPath))); - Assert.assertEquals(0L, vfs.getFileStatus(srcPath).getLen()); + new Path(targetTestRoot, "hdfs-user1/hadoop_dir1"); + testRegexMountpoint( + new Configuration(conf), + regexStr, dstPathStr, interceptors, + srcPath, expectedResolveResult, 2); + } } From 2f3ae037e3533bc6512d8b497683d36cd559f7ad Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Tue, 18 Aug 2020 13:09:21 -0700 Subject: [PATCH 03/12] HADOOP-15891. remove inode tree mounpoints resolution cache. --- .../apache/hadoop/fs/viewfs/Constants.java | 8 - .../apache/hadoop/fs/viewfs/InodeTree.java | 226 +++++++----------- .../fs/viewfs/ViewFileSystemBaseTest.java | 44 ---- .../hadoop-hdfs/src/site/markdown/ViewFs.md | 9 - 4 files changed, 80 insertions(+), 207 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index 947ec3f39ee17..e2f2f9e948ac7 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -126,12 +126,4 @@ public interface Constants { "fs.viewfs.ignore.port.in.mount.table.name"; boolean CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME_DEFAULT = false; - - /** - * Config capacity of mount point resolution cache. - * Value <= 0 means disable cache. - */ - String CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY - = "fs.viewfs.path.resolution.cache.capacity"; - int CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY_DEFAULT = 0; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 6f835b4dfc97e..35b8643da7ecf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -82,9 +82,6 @@ enum ResultKind { private List> mountPoints = new ArrayList>(); private List> regexMountPointList = new ArrayList>(); - private LRUMap pathResolutionCache; - private ReentrantReadWriteLock cacheRWLock; - private int pathResolutionCacheCapacity; static class MountPoint { String src; @@ -686,13 +683,6 @@ protected InodeTree(final Configuration config, final String viewName, theUri); getRootDir().addFallbackLink(rootFallbackLink); } - pathResolutionCacheCapacity = config - .getInt(Constants.CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY, - Constants.CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY_DEFAULT); - if (pathResolutionCacheCapacity > 0) { - pathResolutionCache = new LRUMap(pathResolutionCacheCapacity); - cacheRWLock = new ReentrantReadWriteLock(); - } } /** @@ -736,112 +726,101 @@ boolean isInternalDir() { ResolveResult resolve(final String p, final boolean resolveLastComponent) throws FileNotFoundException { ResolveResult resolveResult = null; - resolveResult = getResolveResultFromCache(p, resolveLastComponent); - if (resolveResult != null) { + String[] path = breakIntoPathComponents(p); + if (path.length <= 1) { // special case for when path is "/" + T targetFs = root.isInternalDir() ? + getRootDir().getInternalDirFs() + : getRootLink().getTargetFileSystem(); + resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, + targetFs, root.fullPath, SlashPath); return resolveResult; } - try { - String[] path = breakIntoPathComponents(p); - if (path.length <= 1) { // special case for when path is "/" - T targetFs = root.isInternalDir() ? - getRootDir().getInternalDirFs() - : getRootLink().getTargetFileSystem(); - resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, - targetFs, root.fullPath, SlashPath); - return resolveResult; - } - - /** - * linkMergeSlash has been configured. The root of this mount table has - * been linked to the root directory of a file system. - * The first non-slash path component should be name of the mount table. - */ - if (root.isLink()) { - Path remainingPath; - StringBuilder remainingPathStr = new StringBuilder(); - // ignore first slash - for (int i = 1; i < path.length; i++) { - remainingPathStr.append("/").append(path[i]); - } - remainingPath = new Path(remainingPathStr.toString()); - resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, - getRootLink().getTargetFileSystem(), root.fullPath, remainingPath); - return resolveResult; - } - Preconditions.checkState(root.isInternalDir()); - INodeDir curInode = getRootDir(); - - // Try to resolve path in the regex mount point - resolveResult = tryResolveInRegexMountpoint(p, resolveLastComponent); - if (resolveResult != null) { - return resolveResult; + /** + * linkMergeSlash has been configured. The root of this mount table has + * been linked to the root directory of a file system. + * The first non-slash path component should be name of the mount table. + */ + if (root.isLink()) { + Path remainingPath; + StringBuilder remainingPathStr = new StringBuilder(); + // ignore first slash + for (int i = 1; i < path.length; i++) { + remainingPathStr.append("/").append(path[i]); } + remainingPath = new Path(remainingPathStr.toString()); + resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, + getRootLink().getTargetFileSystem(), root.fullPath, remainingPath); + return resolveResult; + } + Preconditions.checkState(root.isInternalDir()); + INodeDir curInode = getRootDir(); - int i; - // ignore first slash - for (i = 1; i < path.length - (resolveLastComponent ? 0 : 1); i++) { - INode nextInode = curInode.resolveInternal(path[i]); - if (nextInode == null) { - if (hasFallbackLink()) { - resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, - getRootFallbackLink().getTargetFileSystem(), root.fullPath, - new Path(p)); - return resolveResult; - } else { - StringBuilder failedAt = new StringBuilder(path[0]); - for (int j = 1; j <= i; ++j) { - failedAt.append('/').append(path[j]); - } - throw (new FileNotFoundException( - "File/Directory does not exist: " + failedAt.toString())); - } - } + // Try to resolve path in the regex mount point + resolveResult = tryResolveInRegexMountpoint(p, resolveLastComponent); + if (resolveResult != null) { + return resolveResult; + } - if (nextInode.isLink()) { - final INodeLink link = (INodeLink) nextInode; - final Path remainingPath; - if (i >= path.length - 1) { - remainingPath = SlashPath; - } else { - StringBuilder remainingPathStr = - new StringBuilder("/" + path[i + 1]); - for (int j = i + 2; j < path.length; ++j) { - remainingPathStr.append('/').append(path[j]); - } - remainingPath = new Path(remainingPathStr.toString()); - } + int i; + // ignore first slash + for (i = 1; i < path.length - (resolveLastComponent ? 0 : 1); i++) { + INode nextInode = curInode.resolveInternal(path[i]); + if (nextInode == null) { + if (hasFallbackLink()) { resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, - link.getTargetFileSystem(), nextInode.fullPath, remainingPath); + getRootFallbackLink().getTargetFileSystem(), root.fullPath, + new Path(p)); return resolveResult; - } else if (nextInode.isInternalDir()) { - curInode = (INodeDir) nextInode; + } else { + StringBuilder failedAt = new StringBuilder(path[0]); + for (int j = 1; j <= i; ++j) { + failedAt.append('/').append(path[j]); + } + throw (new FileNotFoundException( + "File/Directory does not exist: " + failedAt.toString())); } } - // We have resolved to an internal dir in mount table. - Path remainingPath; - if (resolveLastComponent) { - remainingPath = SlashPath; - } else { - // note we have taken care of when path is "/" above - // for internal dirs rem-path does not start with / since the lookup - // that follows will do a children.get(remaningPath) and will have to - // strip-out the initial / - StringBuilder remainingPathStr = new StringBuilder("/" + path[i]); - for (int j = i + 1; j < path.length; ++j) { - remainingPathStr.append('/').append(path[j]); + if (nextInode.isLink()) { + final INodeLink link = (INodeLink) nextInode; + final Path remainingPath; + if (i >= path.length - 1) { + remainingPath = SlashPath; + } else { + StringBuilder remainingPathStr = + new StringBuilder("/" + path[i + 1]); + for (int j = i + 2; j < path.length; ++j) { + remainingPathStr.append('/').append(path[j]); + } + remainingPath = new Path(remainingPathStr.toString()); } - remainingPath = new Path(remainingPathStr.toString()); + resolveResult = new ResolveResult(ResultKind.EXTERNAL_DIR, + link.getTargetFileSystem(), nextInode.fullPath, remainingPath); + return resolveResult; + } else if (nextInode.isInternalDir()) { + curInode = (INodeDir) nextInode; } - resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, - curInode.getInternalDirFs(), curInode.fullPath, remainingPath); - return resolveResult; - } finally { - if (pathResolutionCacheCapacity > 0 && resolveResult != null) { - addResolveResultToCache(p, resolveLastComponent, resolveResult); + } + + // We have resolved to an internal dir in mount table. + Path remainingPath; + if (resolveLastComponent) { + remainingPath = SlashPath; + } else { + // note we have taken care of when path is "/" above + // for internal dirs rem-path does not start with / since the lookup + // that follows will do a children.get(remaningPath) and will have to + // strip-out the initial / + StringBuilder remainingPathStr = new StringBuilder("/" + path[i]); + for (int j = i + 1; j < path.length; ++j) { + remainingPathStr.append('/').append(path[j]); } + remainingPath = new Path(remainingPathStr.toString()); } + resolveResult = new ResolveResult(ResultKind.INTERNAL_DIR, + curInode.getInternalDirFs(), curInode.fullPath, remainingPath); + return resolveResult; } /** @@ -906,51 +885,6 @@ protected ResolveResult buildResolveResultForRegexMountPoint( } } - /** - * Return resolution cache capacity. - * - * @return - */ - public int getPathResolutionCacheCapacity() { - return pathResolutionCacheCapacity; - } - - private void addResolveResultToCache(final String pathStr, - final Boolean resolveLastComponent, - final ResolveResult resolveResult) { - try { - String key = getResolveCacheKeyStr(pathStr, resolveLastComponent); - cacheRWLock.writeLock().lock(); - pathResolutionCache.put(key, resolveResult); - } finally { - cacheRWLock.writeLock().unlock(); - } - } - - private ResolveResult getResolveResultFromCache(final String pathStr, - final Boolean resolveLastComponent) { - if (pathResolutionCacheCapacity <= 0) { - return null; - } - try { - String key = getResolveCacheKeyStr(pathStr, resolveLastComponent); - cacheRWLock.readLock().lock(); - return (ResolveResult) pathResolutionCache.get(key); - } finally { - cacheRWLock.readLock().unlock(); - } - } - - public static String getResolveCacheKeyStr(final String path, - Boolean resolveLastComp) { - return path + ",resolveLastComp" + resolveLastComp; - } - - @VisibleForTesting - public LRUMap getPathResolutionCache() { - return pathResolutionCache; - } - List> getMountPoints() { return mountPoints; } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index 26895926037e1..61a67001579f8 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -1431,48 +1431,4 @@ public void testGetContentSummaryWithFileInLocalFS() throws Exception { summaryAfter.getLength()); } } - - @Test - public void testMountPointCache() throws Exception { - conf.setInt(Constants.CONFIG_VIEWFS_PATH_RESOLUTION_CACHE_CAPACITY, 1); - FileSystem fileSystem = FileSystem.get(FsConstants.VIEWFS_URI, conf); - ViewFileSystem viewfs = (ViewFileSystem) fileSystem; - Path resolvedPath1 = new Path(targetTestRoot, "dir3/file1"); - Path srcPath1 = new Path("/internalDir/internalDir2/linkToDir3/file1"); - LRUMap cacheMap = viewfs.fsState.getPathResolutionCache(); - FileSystemTestHelper.createFile(fsTarget, resolvedPath1); - Assert.assertTrue( - resolvedPath1.toString().equals( - fileSystem.resolvePath(srcPath1).toString())); - Assert.assertEquals(viewfs.fsState.getPathResolutionCacheCapacity(), 1); - Assert.assertEquals(viewfs.fsState.getPathResolutionCache().size(), 1); - Assert.assertTrue(viewfs.fsState.getPathResolutionCache().isFull()); - InodeTree.ResolveResult resolveResult1 = - viewfs.fsState.resolve(viewfs.getUriPath(srcPath1), true); - LOG.info("Resolve result resolve path:" + resolveResult1.resolvedPath + - ", remaining path:" + resolveResult1.remainingPath - + ", file kind:" + resolveResult1.kind); - Assert.assertTrue(resolveResult1.resolvedPath.toString().equals( - new Path("/internalDir/internalDir2/linkToDir3").toString())); - Assert.assertTrue(resolveResult1.remainingPath.toString().equals( - "/file1")); - Assert.assertEquals(resolveResult1, - cacheMap.get( - InodeTree.getResolveCacheKeyStr( - viewfs.getUriPath(srcPath1), true))); - - Path srcPath2 = new Path("/internalDir/internalDir2/linkToDir3/file2"); - InodeTree.ResolveResult resolveResult2 = - viewfs.fsState.resolve(viewfs.getUriPath(srcPath2), true); - Assert.assertTrue(resolveResult2.resolvedPath.toString().equals( - new Path("/internalDir/internalDir2/linkToDir3").toString())); - Assert.assertTrue(resolveResult2.remainingPath.toString().equals( - "/file2")); - Assert.assertEquals(viewfs.fsState.getPathResolutionCache().size(), 1); - Assert.assertTrue(viewfs.fsState.getPathResolutionCache().isFull()); - Assert.assertEquals(resolveResult2, - cacheMap.get( - InodeTree.getResolveCacheKeyStr( - viewfs.getUriPath(srcPath2), true))); - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md index 571cb49200220..71cedfef3e734 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md @@ -428,15 +428,6 @@ fs.viewfs.mounttable.${VIEWNAME}.linkRegx.${REGEX_STR} fs.viewfs.mounttable.${VIEWNAME}.linkRegx.${interceptorSettings}#.${srcRegex} ``` -### Path Resolving Cache -Regex parsing time could vary based on the defined rule. It’s possible that the parsing time could be long. So a resolving result cache should definitely help to reduce the parsing time. Besides, the cache could also help reduce the target file system initialization time. An LRU cache is used to implement it. The cache is disabled by default. Please config the following properties to use it. -```xml - - fs.viewfs.path.resolution.cache.capacity - ${cache_cap} - -``` - Appendix: A Mount Table Configuration Example From d540c6b0ffa00351f7e232b72a790f7bb52a8a07 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Tue, 18 Aug 2020 15:28:31 -0700 Subject: [PATCH 04/12] HADOOP-15891. address comments. --- .../apache/hadoop/fs/viewfs/Constants.java | 2 +- .../apache/hadoop/fs/viewfs/InodeTree.java | 3 -- .../hadoop/fs/viewfs/RegexMountPoint.java | 17 ------ .../RegexMountPointInterceptorFactory.java | 2 +- ...ointResolvedDstPathReplaceInterceptor.java | 30 ++++++----- ...ointResolvedDstPathReplaceInterceptor.java | 21 ++++---- .../fs/viewfs/ViewFileSystemBaseTest.java | 1 - .../viewfs/TestViewFileSystemLinkRegex.java | 53 +++++++++---------- 8 files changed, 52 insertions(+), 77 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index e2f2f9e948ac7..21ab8e1aa4cf0 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -100,7 +100,7 @@ public interface Constants { /** * Enable ViewFileSystem to cache all children filesystems in inner cache. - * It won't work for regex based mount points now. + * The children filesystems generated by regex mount points won't be cached. */ String CONFIG_VIEWFS_ENABLE_INNER_CACHE = "fs.viewfs.enable.inner.cache"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 35b8643da7ecf..3bf78cebd72bf 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -17,7 +17,6 @@ */ package org.apache.hadoop.fs.viewfs; -import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import java.io.FileNotFoundException; import java.io.IOException; @@ -30,9 +29,7 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; -import java.util.concurrent.locks.ReentrantReadWriteLock; -import org.apache.commons.collections.map.LRUMap; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java index e5d6c16d5aef1..aace7a2dba57d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java @@ -286,21 +286,4 @@ private String getRegexGroupValueFromMather( return null; } - /** - * Convert interceptor to string. - * - * @param interceptorList - the interceptor list to be applied. - * @return - */ - public static String convertInterceptorsToString( - List interceptorList) { - StringBuffer stringBuffer = new StringBuffer(); - for (int index = 0; index < interceptorList.size(); ++index) { - stringBuffer.append(interceptorList.get(index).toString()); - if (index < interceptorList.size() - 1) { - stringBuffer.append(INTERCEPTOR_SEP); - } - } - return stringBuffer.toString(); - } } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java index d9b3f54021333..fb564aa3a6e4d 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointInterceptorFactory.java @@ -36,7 +36,7 @@ private RegexMountPointInterceptorFactory() { * e.g. replaceresolveddstpath:word1,word2. * * @param interceptorSettingsString - * @return + * @return Return interceptor based on setting or null on bad/unknown config. */ public static RegexMountPointInterceptor create( String interceptorSettingsString) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java index d89b93df2b973..16c578f910d64 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java @@ -59,7 +59,8 @@ public Pattern getSrcRegexPattern() { return srcRegexPattern; } - @Override public void initialize() throws IOException { + @Override + public void initialize() throws IOException { try { srcRegexPattern = Pattern.compile(srcRegexString); } catch (PatternSyntaxException ex) { @@ -69,12 +70,12 @@ public Pattern getSrcRegexPattern() { } /** - * Intercept source before resolution. + * Source won't be changed in the interceptor. * - * @param source - * @return + * @return source param string passed in. */ - @Override public String interceptSource(String source) { + @Override + public String interceptSource(String source) { return source; } @@ -86,26 +87,30 @@ public Pattern getSrcRegexPattern() { * * @return intercepted string */ - @Override public String interceptResolvedDestPathStr( + @Override + public String interceptResolvedDestPathStr( String parsedDestPathStr) { Matcher matcher = srcRegexPattern.matcher(parsedDestPathStr); return matcher.replaceAll(replaceString); } /** - * Intercept remaining path. + * The interceptRemainingPath will just return the remainingPath passed in. * - * @return intercepted path + * @return remaining path pass in */ - @Override public Path interceptRemainingPath(Path remainingPath) { + @Override + public Path interceptRemainingPath(Path remainingPath) { return remainingPath; } - @Override public RegexMountPointInterceptorType getType() { + @Override + public RegexMountPointInterceptorType getType() { return REPLACE_RESOLVED_DST_PATH; } - @Override public String serializeToString() { + @Override + public String serializeToString() { return REPLACE_RESOLVED_DST_PATH.getConfigName() + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + srcRegexString + RegexMountPoint.INTERCEPTOR_INTERNAL_SEP + replaceString; @@ -116,8 +121,7 @@ public Pattern getSrcRegexPattern() { * replaceresolvedpath:wordToReplace:replaceString * Note that we'll assume there's no ':' in the regex for the moment. * - * @param serializedString - * @return + * @return Interceptor instance or null on bad config. */ public static RegexMountPointResolvedDstPathReplaceInterceptor deserializeFromString(String serializedString) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java index 598d5c759d95d..9fdf0f6ac9c5c 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java @@ -19,7 +19,6 @@ import java.io.IOException; -import org.apache.hadoop.fs.Path; import org.junit.Assert; import org.junit.Test; @@ -44,12 +43,12 @@ public void testDeserializeFromStringNormalCase() throws IOException { RegexMountPointResolvedDstPathReplaceInterceptor interceptor = RegexMountPointResolvedDstPathReplaceInterceptor .deserializeFromString(serializedString); - Assert.assertTrue(interceptor.getSrcRegexString().equals(srcRegex)); - Assert.assertTrue(interceptor.getReplaceString().equals(replaceString)); - Assert.assertTrue(interceptor.getSrcRegexPattern() == null); + Assert.assertEquals(srcRegex, interceptor.getSrcRegexString()); + Assert.assertEquals(replaceString, interceptor.getReplaceString()); + Assert.assertNull(interceptor.getSrcRegexPattern()); interceptor.initialize(); - Assert.assertTrue( - interceptor.getSrcRegexPattern().toString().equals(srcRegex)); + Assert.assertEquals(srcRegex, + interceptor.getSrcRegexPattern().toString()); } @Test @@ -61,7 +60,7 @@ public void testDeserializeFromStringBadCase() throws IOException { RegexMountPointResolvedDstPathReplaceInterceptor interceptor = RegexMountPointResolvedDstPathReplaceInterceptor .deserializeFromString(serializedString); - Assert.assertEquals(interceptor, null); + Assert.assertNull(interceptor); } @Test @@ -83,13 +82,12 @@ public void testInterceptSource() { new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, replaceString); String sourcePath = "/a/b/l3/dd"; - sourcePath = interceptor.interceptSource(sourcePath); + Assert.assertEquals(sourcePath, interceptor.interceptSource(sourcePath)); } @Test public void testInterceptResolve() throws IOException { String pathAfterResolution = "/user-hadoop"; - Path remainingPath = new Path("/ad-data"); String srcRegex = "hadoop"; String replaceString = "hdfs"; @@ -97,8 +95,7 @@ public void testInterceptResolve() throws IOException { new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex, replaceString); interceptor.initialize(); - Assert.assertTrue( - interceptor.interceptResolvedDestPathStr(pathAfterResolution) - .equals("/user-hdfs")); + Assert.assertEquals("/user-hdfs", + interceptor.interceptResolvedDestPathStr(pathAfterResolution)); } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java index 61a67001579f8..05d7974395013 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java @@ -31,7 +31,6 @@ import java.util.Map.Entry; import java.util.Random; -import org.apache.commons.collections.map.LRUMap; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.BlockStoragePolicySpi; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java index c341e8fb8b111..61aa7a4542358 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -246,16 +246,15 @@ private void testRegexMountpoint( */ @Test public void testConfLinkRegexIndexMapping() throws Exception { - // CHECKSTYLE:OFF // Config: // // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.^/(\w+) // /targetTestRoot/$1 // // Dir path to test: /testConfLinkRegexIndexMapping1 // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping1 - // CHECKSTYLE:ON String regexStr = "^/(\\w+)"; String dstPathStr = targetTestRoot + "$1"; Path srcPath = new Path("/testConfLinkRegexIndexMapping1"); @@ -266,16 +265,15 @@ public void testConfLinkRegexIndexMapping() throws Exception { regexStr, dstPathStr, null, srcPath, expectedResolveResult, 3); - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.^/(\w+) // /targetTestRoot/${1} // // Dir path to test: /testConfLinkRegexIndexMapping2 // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping2 - // CHECKSTYLE:ON dstPathStr = targetTestRoot + "${1}"; srcPath = new Path("/testConfLinkRegexIndexMapping2"); expectedResolveResult = @@ -286,15 +284,14 @@ public void testConfLinkRegexIndexMapping() throws Exception { regexStr, dstPathStr, null, srcPath, expectedResolveResult, 4); - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.^/(\w+) // /targetTestRoot/$1 // // Dir path to test: /testConfLinkRegexIndexMapping3/dir1 // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping3/dir1 - // CHECKSTYLE:ON dstPathStr = targetTestRoot + "$1"; srcPath = new Path("/testConfLinkRegexIndexMapping3/dir1"); expectedResolveResult = new Path( @@ -304,15 +301,14 @@ public void testConfLinkRegexIndexMapping() throws Exception { regexStr, dstPathStr, null, srcPath, expectedResolveResult, 5); - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.^/(\w+) // /targetTestRoot/${1}/ // // Dir path to test: /testConfLinkRegexIndexMapping4/dir1 // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping4/dir1 - // CHECKSTYLE:ON dstPathStr = targetTestRoot + "${1}/"; srcPath = new Path("/testConfLinkRegexIndexMapping4/dir1"); expectedResolveResult = new Path( @@ -329,15 +325,14 @@ public void testConfLinkRegexIndexMapping() throws Exception { */ @Test public void testConfLinkRegexNamedGroupMapping() throws Exception { - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(?\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.^/(?\w+) // /targetTestRoot/$firstDir // // Dir path to test: /testConfLinkRegexNamedGroupMapping1 // Expect path: /targetTestRoot/testConfLinkRegexNamedGroupMapping1 - // CHECKSTYLE:ON URI viewFsUri = new URI( FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); String regexStr = "^/(?\\w+)"; @@ -350,19 +345,19 @@ public void testConfLinkRegexNamedGroupMapping() throws Exception { regexStr, dstPathStr, null, srcPath, expectedResolveResult, 3); - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(?\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.^/(?\w+) // /targetTestRoot/${firstDir} // // Dir path to test: /testConfLinkRegexNamedGroupMapping2 // Expect path: /targetTestRoot/testConfLinkRegexNamedGroupMapping2 - // CHECKSTYLE:ON dstPathStr = targetTestRoot + "${firstDir}"; srcPath = new Path("/testConfLinkRegexNamedGroupMapping2"); expectedResolveResult = new Path( - dstPathStr.replace("${firstDir}", "testConfLinkRegexNamedGroupMapping2")); + dstPathStr.replace( + "${firstDir}", "testConfLinkRegexNamedGroupMapping2")); testRegexMountpoint( new Configuration(conf), regexStr, dstPathStr, null, @@ -375,17 +370,16 @@ public void testConfLinkRegexNamedGroupMapping() throws Exception { */ @Test public void testConfLinkRegexFixedDestMapping() throws Exception { - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.^/(?\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.^/(?\w+) // /targetTestRoot/${firstDir} // // Dir path to test: /misc1 // Expect path: /targetTestRoot/testConfLinkRegexFixedDestMappingFile // Dir path to test: /misc2 // Expect path: /targetTestRoot/testConfLinkRegexFixedDestMappingFile - // CHECKSTYLE:ON String regexStr = "^/\\w+"; String dstPathStr = targetTestRoot + "testConfLinkRegexFixedDestMappingFile"; @@ -406,15 +400,14 @@ public void testConfLinkRegexFixedDestMapping() throws Exception { */ @Test public void testConfLinkRegexWithSingleInterceptor() throws Exception { - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.replaceresolveddstpath:_:-#.^/user/(?\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex.replaceresolveddstpath:_:-#.^/user/(?\w+) // /targetTestRoot/$username // // Dir path to test: /user/hadoop_user1/hadoop_dir1 // Expect path: /targetTestRoot/hadoop-user1/hadoop_dir1 - // CHECKSTYLE:ON String regexStr = "^/user/(?\\w+)"; String dstPathStr = targetTestRoot + "$username"; @@ -435,17 +428,19 @@ public void testConfLinkRegexWithSingleInterceptor() throws Exception { */ @Test public void testConfLinkRegexWithInterceptors() throws Exception { - // CHECKSTYLE:OFF // Config: // - // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster.linkRegex.replaceresolveddstpath:_:-;replaceresolveddstpath:hadoop:hdfs#.^/user/(?\w+) + // fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster + // .linkRegex + // .replaceresolveddstpath:_:-; + // replaceresolveddstpath:hadoop:hdfs#.^/user/(?\w+) // /targetTestRoot/$username // // Dir path to test: /user/hadoop_user1/hadoop_dir1 // Expect path: /targetTestRoot/hdfs-user1/hadoop_dir1 - // CHECKSTYLE:ON URI viewFsUri = - new URI(FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + new URI( + FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); String regexStr = "^/user/(?\\w+)/"; String dstPathStr = targetTestRoot + "$username"; // Replace "_" with "-" From c609d13f77950d082525bb06a931540fe9119cb9 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Sat, 5 Sep 2020 21:04:01 -0700 Subject: [PATCH 05/12] HADOOP-15891: address comments. Add inner cache support. --- .../apache/hadoop/fs/viewfs/Constants.java | 8 +++ .../apache/hadoop/fs/viewfs/InodeTree.java | 11 +--- .../hadoop/fs/viewfs/ViewFileSystem.java | 58 +++++++++++++------ .../org/apache/hadoop/fs/viewfs/ViewFs.java | 8 --- .../hadoop/fs/viewfs/TestRegexMountPoint.java | 6 -- .../hadoop/fs/viewfs/TestViewFsConfig.java | 6 -- .../hadoop-hdfs/src/site/markdown/ViewFs.md | 2 +- .../viewfs/TestViewFileSystemLinkRegex.java | 17 +++++- 8 files changed, 65 insertions(+), 51 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index 21ab8e1aa4cf0..e84bd0748ed88 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -106,6 +106,14 @@ public interface Constants { boolean CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT = true; + /** + * Whether to evict cached children file times on ViewFileSystem::close() + */ + String CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE = + "fs.viewfs.inner.cache.evict.on.close"; + + boolean CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE_DEFAULT = false; + /** * Enable ViewFileSystem to show mountlinks as symlinks. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 3bf78cebd72bf..73bbac2e93e36 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -388,9 +388,6 @@ private void createLink(final String src, final String target, protected abstract T getTargetFileSystem(URI uri) throws UnsupportedFileSystemException, URISyntaxException, IOException; - protected abstract T getTargetFileSystem(URI uri, boolean enableCache) - throws UnsupportedFileSystemException, URISyntaxException, IOException; - protected abstract T getTargetFileSystem(INodeDir dir) throws URISyntaxException, IOException; @@ -847,18 +844,14 @@ protected ResolveResult tryResolveInRegexMountpoint(final String srcPath, /** * Build resolve result return to caller. * - * @param resultKind - * @param resolvedPathStr - * @param targetOfResolvedPathStr - * @param remainingPath - * @return + * @return targetFileSystem or null on exceptions. */ protected ResolveResult buildResolveResultForRegexMountPoint( ResultKind resultKind, String resolvedPathStr, String targetOfResolvedPathStr, Path remainingPath) { try { T targetFs = getTargetFileSystem( - new URI(targetOfResolvedPathStr), false); + new URI(targetOfResolvedPathStr)); return new ResolveResult(resultKind, targetFs, resolvedPathStr, remainingPath); } catch (IOException ex) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index b173271134987..2d285a755d5d3 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -22,6 +22,8 @@ import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME_DEFAULT; +import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE; +import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE_DEFAULT; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT; import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_555; @@ -33,7 +35,6 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; -import java.util.Collections; import java.util.EnumSet; import java.util.HashMap; import java.util.HashSet; @@ -42,6 +43,7 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Set; +import java.util.concurrent.locks.ReentrantReadWriteLock; import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; @@ -114,6 +116,7 @@ protected FsGetter fsGetter() { static class InnerCache { private Map map = new HashMap<>(); private FsGetter fsCreator; + private ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock(); InnerCache(FsGetter fsCreator) { this.fsCreator = fsCreator; @@ -121,12 +124,27 @@ static class InnerCache { FileSystem get(URI uri, Configuration config) throws IOException { Key key = new Key(uri); - if (map.get(key) == null) { - FileSystem fs = fsCreator.getNewInstance(uri, config); + FileSystem fs = null; + try { + rwLock.readLock().lock(); + fs = map.get(key); + if (fs != null) { + return fs; + } + } finally { + rwLock.readLock().unlock(); + } + try { + rwLock.writeLock().lock(); + fs = map.get(key); + if (fs != null) { + return fs; + } + fs = fsCreator.getNewInstance(uri, config); map.put(key, fs); return fs; - } else { - return map.get(key); + } finally { + rwLock.writeLock().unlock(); } } @@ -140,9 +158,13 @@ void closeAll() { } } - InnerCache unmodifiableCache() { - map = Collections.unmodifiableMap(map); - return this; + void clear() { + try { + rwLock.writeLock().lock(); + map.clear(); + } finally { + rwLock.writeLock().unlock(); + } } /** @@ -217,6 +239,7 @@ public URI[] getTargetFileSystemURIs() { Path homeDir = null; private boolean enableInnerCache = false; private InnerCache cache; + private boolean evictCacheOnClose = false; // Default to rename within same mountpoint private RenameStrategy renameStrategy = RenameStrategy.SAME_MOUNTPOINT; /** @@ -281,6 +304,9 @@ public void initialize(final URI theUri, final Configuration conf) config = conf; enableInnerCache = config.getBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT); + evictCacheOnClose = config.getBoolean( + CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE, + CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE_DEFAULT); FsGetter fsGetter = fsGetter(); final InnerCache innerCache = new InnerCache(fsGetter); // Now build client side view (i.e. client side mount table) from config. @@ -300,15 +326,8 @@ public void initialize(final URI theUri, final Configuration conf) @Override protected FileSystem getTargetFileSystem(final URI uri) throws URISyntaxException, IOException { - return getTargetFileSystem(uri, enableInnerCache); - } - - @Override - protected FileSystem getTargetFileSystem( - final URI uri, boolean enableCache) - throws URISyntaxException, IOException { FileSystem fs; - if (enableCache) { + if (enableInnerCache) { fs = innerCache.get(uri, config); } else { fs = fsGetter.get(uri, config); @@ -342,7 +361,7 @@ protected FileSystem getTargetFileSystem(final String settings, // All fs instances are created and cached on startup. The cache is // readonly after the initialize() so the concurrent access of the cache // is safe. - cache = innerCache.unmodifiableCache(); + cache = innerCache; } } @@ -414,7 +433,7 @@ public FSDataOutputStream append(final Path f, final int bufferSize, fsState.resolve(getUriPath(f), true); return res.targetFileSystem.append(res.remainingPath, bufferSize, progress); } - + @Override public FSDataOutputStream createNonRecursive(Path f, FsPermission permission, EnumSet flags, int bufferSize, short replication, @@ -1697,6 +1716,9 @@ public void close() throws IOException { super.close(); if (enableInnerCache && cache != null) { cache.closeAll(); + if (evictCacheOnClose) { + cache.clear(); + } } } } 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 d3d6373d27596..95b596bde367d 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 @@ -248,14 +248,6 @@ protected AbstractFileSystem getTargetFileSystem(final URI uri) new Path(pathString)); } - @Override - protected AbstractFileSystem getTargetFileSystem( - final URI uri, boolean enableCache) - throws URISyntaxException, UnsupportedFileSystemException { - throw new UnsupportedFileSystemException( - "Cache option is not supported in ViewFs! Uri:" + uri); - } - @Override protected AbstractFileSystem getTargetFileSystem( final INodeDir dir) throws URISyntaxException { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java index 34b3b8fdf4b4e..df71f8ea77fce 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java @@ -68,12 +68,6 @@ protected TestRegexMountPointFileSystem getTargetFileSystem( return new TestRegexMountPointFileSystem(uri); } - @Override - protected TestRegexMountPointFileSystem getTargetFileSystem( - final URI uri, boolean enableCache) { - return new TestRegexMountPointFileSystem(uri); - } - @Override protected TestRegexMountPointFileSystem getTargetFileSystem( final INodeDir dir) { diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java index ec1a97bf3b86d..56f5b2d997dc2 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsConfig.java @@ -46,12 +46,6 @@ protected Foo getTargetFileSystem(final URI uri) { return null; } - @Override - protected Foo getTargetFileSystem( - final URI uri, boolean enableCache) { - return null; - } - @Override protected Foo getTargetFileSystem(final INodeDir dir) { return null; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md index 71cedfef3e734..0c88505a84f9d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md @@ -378,7 +378,7 @@ The view file system mount points were a Key-Value based mapping system. It is n ### Understand the Difference In the key-value based mount table, view file system treats every mount point as a partition. There's several file system APIs which will lead to operation on all partitions. E.g. there's an HDFS cluster with multiple mount. Users want to run “hadoop fs -put file viewfs://hdfs.namenode.apache.org/tmp/” cmd to copy data from local disk to our HDFS cluster. The cmd will trigger ViewFileSystem to call setVerifyChecksum() method which will initialize the file system for every mount point. -For a regex-base rule mount table entry, we couldn't know what's corresponding path until parsing. So the regex based mount table entry will be ignored on such cases. The file system will be created upon accessing. The inner cache of ViewFs is also not available for regex-base mount points now as it assumes target file system doesn't change after viewfs initialization. +For a regex-base rule mount table entry, we couldn't know what's corresponding path until parsing. So the regex based mount table entry will be ignored on such cases. The file system (ChRootedFileSystem) will be created upon accessing. But the underlying file system will be cached by inner cache of ViewFileSystem. ```xml fs.viewfs.rename.strategy diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java index 61aa7a4542358..da24832f28019 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -45,6 +45,7 @@ import org.slf4j.LoggerFactory; import static org.apache.hadoop.fs.viewfs.RegexMountPoint.INTERCEPTOR_INTERNAL_SEP; +import static org.junit.Assert.assertSame; /** * Test linkRegex node type for view file system. @@ -233,6 +234,19 @@ private void testRegexMountpoint( vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory()); Assert.assertEquals( childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); + + // Test Inner cache, the resolved result's filesystem should be the same. + ViewFileSystem viewFileSystem = (ViewFileSystem) vfs; + ChRootedFileSystem target1 = (ChRootedFileSystem) viewFileSystem + .fsState.resolve( + viewFileSystem.getUriPath( + dirPathBeforeMountPoint), true).targetFileSystem; + ChRootedFileSystem target2 = (ChRootedFileSystem) viewFileSystem + .fsState.resolve( + viewFileSystem.getUriPath( + dirPathBeforeMountPoint), true).targetFileSystem; + assertSame(target1.getMyFs(), target2.getMyFs()); + } finally { if (vfs != null) { vfs.close(); @@ -438,9 +452,6 @@ public void testConfLinkRegexWithInterceptors() throws Exception { // // Dir path to test: /user/hadoop_user1/hadoop_dir1 // Expect path: /targetTestRoot/hdfs-user1/hadoop_dir1 - URI viewFsUri = - new URI( - FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); String regexStr = "^/user/(?\\w+)/"; String dstPathStr = targetTestRoot + "$username"; // Replace "_" with "-" From e25945d5da8ecd58ba3eb1e9f28715291c4273f1 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Sat, 5 Sep 2020 22:50:25 -0700 Subject: [PATCH 06/12] Add more comments. --- .../org/apache/hadoop/fs/viewfs/InodeTree.java | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 73bbac2e93e36..89ad67fff3c59 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -842,7 +842,20 @@ protected ResolveResult tryResolveInRegexMountpoint(final String srcPath, } /** - * Build resolve result return to caller. + * Build resolve result. + * Here's an example + * Config: + * + * fs.viewfs.mounttable.mt + * .linkRegex.replaceresolveddstpath:_:-#.^/user/(?\w+) + * /targetTestRoot/$username + * + * Dir path to test: + * viewfs://mt/user/hadoop_user1/hadoop_dir1 + * Expect path: /targetTestRoot/hadoop-user1/hadoop_dir1 + * resolvedPathStr: /user/hadoop_user1 + * targetOfResolvedPathStr: /targetTestRoot/hadoop-user1 + * remainingPath: /hadoop_dir1 * * @return targetFileSystem or null on exceptions. */ From 8786ee8fc0e1b1d617eabd398c4d05a1ff656002 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Tue, 8 Sep 2020 13:43:56 -0700 Subject: [PATCH 07/12] Fix checkstyle and rebase. --- .../main/java/org/apache/hadoop/fs/viewfs/Constants.java | 2 +- .../main/java/org/apache/hadoop/fs/viewfs/InodeTree.java | 9 +++------ 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index e84bd0748ed88..deaefe2988460 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -107,7 +107,7 @@ public interface Constants { boolean CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT = true; /** - * Whether to evict cached children file times on ViewFileSystem::close() + * Whether to evict cached children file times on ViewFileSystem::close(). */ String CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE = "fs.viewfs.inner.cache.evict.on.close"; diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 89ad67fff3c59..2af4dc150d9e8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -844,12 +844,9 @@ protected ResolveResult tryResolveInRegexMountpoint(final String srcPath, /** * Build resolve result. * Here's an example - * Config: - * - * fs.viewfs.mounttable.mt - * .linkRegex.replaceresolveddstpath:_:-#.^/user/(?\w+) - * /targetTestRoot/$username - * + * Mountpoint: fs.viewfs.mounttable.mt + * .linkRegex.replaceresolveddstpath:_:-#.^/user/(?\w+) + * Value: /targetTestRoot/$username * Dir path to test: * viewfs://mt/user/hadoop_user1/hadoop_dir1 * Expect path: /targetTestRoot/hadoop-user1/hadoop_dir1 From 1cb9b50d18cfaba2d2b4dd2fb84913567ab29201 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Tue, 8 Sep 2020 23:00:26 -0700 Subject: [PATCH 08/12] HADOOP-15891: fix comments. --- .../apache/hadoop/fs/viewfs/Constants.java | 7 --- ...ointResolvedDstPathReplaceInterceptor.java | 1 - .../hadoop/fs/viewfs/ViewFileSystem.java | 9 +--- .../hadoop/fs/viewfs/TestRegexMountPoint.java | 18 ++++---- .../hadoop-hdfs/src/site/markdown/ViewFs.md | 2 +- .../viewfs/TestViewFileSystemLinkRegex.java | 44 +++++++------------ 6 files changed, 27 insertions(+), 54 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index deaefe2988460..d6015549379cd 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -100,18 +100,11 @@ public interface Constants { /** * Enable ViewFileSystem to cache all children filesystems in inner cache. - * The children filesystems generated by regex mount points won't be cached. */ String CONFIG_VIEWFS_ENABLE_INNER_CACHE = "fs.viewfs.enable.inner.cache"; boolean CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT = true; - /** - * Whether to evict cached children file times on ViewFileSystem::close(). - */ - String CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE = - "fs.viewfs.inner.cache.evict.on.close"; - boolean CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE_DEFAULT = false; /** diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java index 16c578f910d64..18490dc57c5ac 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java @@ -97,7 +97,6 @@ public String interceptResolvedDestPathStr( /** * The interceptRemainingPath will just return the remainingPath passed in. * - * @return remaining path pass in */ @Override public Path interceptRemainingPath(Path remainingPath) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index 2d285a755d5d3..e14b97236b46f 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -22,8 +22,6 @@ import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_IGNORE_PORT_IN_MOUNT_TABLE_NAME_DEFAULT; -import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE; -import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE_DEFAULT; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS; import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT; import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_555; @@ -304,9 +302,6 @@ public void initialize(final URI theUri, final Configuration conf) config = conf; enableInnerCache = config.getBoolean(CONFIG_VIEWFS_ENABLE_INNER_CACHE, CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT); - evictCacheOnClose = config.getBoolean( - CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE, - CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE_DEFAULT); FsGetter fsGetter = fsGetter(); final InnerCache innerCache = new InnerCache(fsGetter); // Now build client side view (i.e. client side mount table) from config. @@ -1716,9 +1711,7 @@ public void close() throws IOException { super.close(); if (enableInnerCache && cache != null) { cache.closeAll(); - if (evictCacheOnClose) { - cache.clear(); - } + cache.clear(); } } } diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java index df71f8ea77fce..5513b6005b41e 100644 --- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java @@ -119,14 +119,14 @@ public void testResolve() throws IOException { Assert.assertTrue( resolveResult.targetFileSystem instanceof TestRegexMountPointFileSystem); - Assert.assertTrue(resolveResult.resolvedPath.equals("/user/hadoop")); + Assert.assertEquals("/user/hadoop", resolveResult.resolvedPath); Assert.assertTrue( resolveResult.targetFileSystem instanceof TestRegexMountPointFileSystem); - Assert.assertTrue( + Assert.assertEquals("/namenode1/testResolve/hadoop", ((TestRegexMountPointFileSystem) resolveResult.targetFileSystem) - .getUri().toString().equals("/namenode1/testResolve/hadoop")); - Assert.assertTrue(resolveResult.remainingPath.toString().equals("/file1")); + .getUri().toString()); + Assert.assertEquals("/file1", resolveResult.remainingPath.toString()); } @Test @@ -147,14 +147,14 @@ public void testResolveWithInterceptor() throws IOException { Assert.assertTrue( resolveResult.targetFileSystem instanceof TestRegexMountPointFileSystem); - Assert.assertTrue(resolveResult.resolvedPath.equals("/user/hadoop_user1")); + Assert.assertEquals("/user/hadoop_user1", resolveResult.resolvedPath); Assert.assertTrue( resolveResult.targetFileSystem instanceof TestRegexMountPointFileSystem); - Assert.assertTrue( + Assert.assertEquals("/namenode1/testResolve/hadoop-user1", ((TestRegexMountPointFileSystem) resolveResult.targetFileSystem) - .getUri().toString().equals("/namenode1/testResolve/hadoop-user1")); - Assert.assertTrue( - resolveResult.remainingPath.toString().equals("/file_index")); + .getUri().toString()); + Assert.assertEquals("/file_index", + resolveResult.remainingPath.toString()); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md index 0c88505a84f9d..b29a888475941 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFs.md @@ -378,7 +378,7 @@ The view file system mount points were a Key-Value based mapping system. It is n ### Understand the Difference In the key-value based mount table, view file system treats every mount point as a partition. There's several file system APIs which will lead to operation on all partitions. E.g. there's an HDFS cluster with multiple mount. Users want to run “hadoop fs -put file viewfs://hdfs.namenode.apache.org/tmp/” cmd to copy data from local disk to our HDFS cluster. The cmd will trigger ViewFileSystem to call setVerifyChecksum() method which will initialize the file system for every mount point. -For a regex-base rule mount table entry, we couldn't know what's corresponding path until parsing. So the regex based mount table entry will be ignored on such cases. The file system (ChRootedFileSystem) will be created upon accessing. But the underlying file system will be cached by inner cache of ViewFileSystem. +For a regex rule based mount table entry, we couldn't know what's corresponding path until parsing. So the regex based mount table entry will be ignored on such cases. The file system (ChRootedFileSystem) will be created upon accessing. But the underlying file system will be cached by inner cache of ViewFileSystem. ```xml fs.viewfs.rename.strategy diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java index da24832f28019..c4d6eb82775b6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -215,43 +215,31 @@ private void testRegexMountpoint( Path expectedResolveResult, int childrenFilesCnt) throws IOException, URISyntaxException { - FileSystem vfs = null; - try { // Set up test env createDirWithChildren( fsTarget, expectedResolveResult, childrenFilesCnt); ConfigUtil.addLinkRegex( config, CLUSTER_NAME, regexStr, dstPathStr, interceptorSettings); - // Asserts URI viewFsUri = new URI( FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); - vfs = FileSystem.get(viewFsUri, config); - Assert.assertTrue( - expectedResolveResult.equals( - vfs.resolvePath(dirPathBeforeMountPoint))); - Assert.assertTrue( - vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory()); - Assert.assertEquals( - childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); - - // Test Inner cache, the resolved result's filesystem should be the same. - ViewFileSystem viewFileSystem = (ViewFileSystem) vfs; - ChRootedFileSystem target1 = (ChRootedFileSystem) viewFileSystem - .fsState.resolve( - viewFileSystem.getUriPath( - dirPathBeforeMountPoint), true).targetFileSystem; - ChRootedFileSystem target2 = (ChRootedFileSystem) viewFileSystem - .fsState.resolve( - viewFileSystem.getUriPath( - dirPathBeforeMountPoint), true).targetFileSystem; - assertSame(target1.getMyFs(), target2.getMyFs()); - - } finally { - if (vfs != null) { - vfs.close(); + try (FileSystem vfs = FileSystem.get(viewFsUri, config)) { + Assert.assertEquals(expectedResolveResult.toString(), + vfs.resolvePath(dirPathBeforeMountPoint).toString()); + Assert.assertTrue( + vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory()); + Assert.assertEquals(childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); + + // Test Inner cache, the resolved result's filesystem should be the same. + ViewFileSystem viewFileSystem = (ViewFileSystem) vfs; + ChRootedFileSystem target1 = (ChRootedFileSystem) viewFileSystem.fsState + .resolve(viewFileSystem.getUriPath(dirPathBeforeMountPoint), true) + .targetFileSystem; + ChRootedFileSystem target2 = (ChRootedFileSystem) viewFileSystem.fsState + .resolve(viewFileSystem.getUriPath(dirPathBeforeMountPoint), true) + .targetFileSystem; + assertSame(target1.getMyFs(), target2.getMyFs()); } - } } /** * Test regex mount points which use capture group index for mapping. From 4c543eae3410afb35553a4701ae911919c1f009c Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Wed, 9 Sep 2020 08:47:07 -0700 Subject: [PATCH 09/12] Fix checkstyle. --- .../apache/hadoop/fs/viewfs/Constants.java | 2 - .../hadoop/fs/viewfs/ViewFileSystem.java | 1 - .../viewfs/TestViewFileSystemLinkRegex.java | 48 +++++++++---------- 3 files changed, 24 insertions(+), 27 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java index d6015549379cd..bf9f7db7223d8 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java @@ -105,8 +105,6 @@ public interface Constants { boolean CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT = true; - boolean CONFIG_VIEWFS_INNER_CACHE_EVICT_ON_CLOSE_DEFAULT = false; - /** * Enable ViewFileSystem to show mountlinks as symlinks. */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java index e14b97236b46f..c7ed15bba9c0b 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java @@ -237,7 +237,6 @@ public URI[] getTargetFileSystemURIs() { Path homeDir = null; private boolean enableInnerCache = false; private InnerCache cache; - private boolean evictCacheOnClose = false; // Default to rename within same mountpoint private RenameStrategy renameStrategy = RenameStrategy.SAME_MOUNTPOINT; /** diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java index c4d6eb82775b6..3907e7447e0be 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -216,30 +216,30 @@ private void testRegexMountpoint( int childrenFilesCnt) throws IOException, URISyntaxException { // Set up test env - createDirWithChildren( - fsTarget, expectedResolveResult, childrenFilesCnt); - ConfigUtil.addLinkRegex( - config, CLUSTER_NAME, regexStr, dstPathStr, interceptorSettings); - // Asserts - URI viewFsUri = new URI( - FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); - try (FileSystem vfs = FileSystem.get(viewFsUri, config)) { - Assert.assertEquals(expectedResolveResult.toString(), - vfs.resolvePath(dirPathBeforeMountPoint).toString()); - Assert.assertTrue( - vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory()); - Assert.assertEquals(childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); - - // Test Inner cache, the resolved result's filesystem should be the same. - ViewFileSystem viewFileSystem = (ViewFileSystem) vfs; - ChRootedFileSystem target1 = (ChRootedFileSystem) viewFileSystem.fsState - .resolve(viewFileSystem.getUriPath(dirPathBeforeMountPoint), true) - .targetFileSystem; - ChRootedFileSystem target2 = (ChRootedFileSystem) viewFileSystem.fsState - .resolve(viewFileSystem.getUriPath(dirPathBeforeMountPoint), true) - .targetFileSystem; - assertSame(target1.getMyFs(), target2.getMyFs()); - } + createDirWithChildren( + fsTarget, expectedResolveResult, childrenFilesCnt); + ConfigUtil.addLinkRegex( + config, CLUSTER_NAME, regexStr, dstPathStr, interceptorSettings); + // Asserts + URI viewFsUri = new URI( + FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null); + try (FileSystem vfs = FileSystem.get(viewFsUri, config)) { + Assert.assertEquals(expectedResolveResult.toString(), + vfs.resolvePath(dirPathBeforeMountPoint).toString()); + Assert.assertTrue( + vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory()); + Assert.assertEquals(childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); + + // Test Inner cache, the resolved result's filesystem should be the same. + ViewFileSystem viewFileSystem = (ViewFileSystem) vfs; + ChRootedFileSystem target1 = (ChRootedFileSystem) viewFileSystem.fsState + .resolve(viewFileSystem.getUriPath(dirPathBeforeMountPoint), true) + .targetFileSystem; + ChRootedFileSystem target2 = (ChRootedFileSystem) viewFileSystem.fsState + .resolve(viewFileSystem.getUriPath(dirPathBeforeMountPoint), true) + .targetFileSystem; + assertSame(target1.getMyFs(), target2.getMyFs()); + } } /** * Test regex mount points which use capture group index for mapping. From fa30c5ec629e54379368db91dcf51f2414b03d28 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Wed, 9 Sep 2020 15:04:54 -0700 Subject: [PATCH 10/12] Fix checkstyle --- .../apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java index 3907e7447e0be..d3afa47f7554b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java @@ -228,7 +228,8 @@ private void testRegexMountpoint( vfs.resolvePath(dirPathBeforeMountPoint).toString()); Assert.assertTrue( vfs.getFileStatus(dirPathBeforeMountPoint).isDirectory()); - Assert.assertEquals(childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); + Assert.assertEquals( + childrenFilesCnt, vfs.listStatus(dirPathBeforeMountPoint).length); // Test Inner cache, the resolved result's filesystem should be the same. ViewFileSystem viewFileSystem = (ViewFileSystem) vfs; From c494bfe25c6c71d813194c3ed4561866db196f37 Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Wed, 9 Sep 2020 22:18:44 -0700 Subject: [PATCH 11/12] Remove useless method. --- .../org/apache/hadoop/fs/viewfs/ConfigUtil.java | 14 -------------- 1 file changed, 14 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java index 91f87448cd342..09ec5d2933058 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ConfigUtil.java @@ -166,20 +166,6 @@ public static void addLinkNfly(final Configuration conf, final String src, addLinkNfly(conf, getDefaultMountTableName(conf), src, null, targets); } - - /** - * Add a LinkRegex to the config for the specified mount table. - * @param conf - get mountable config from this conf - * @param mountTableName - the mountable name of the regex config item - * @param srcRegex - the src path regex expression that applies to this config - * @param targetStr - the string of target path - */ - public static void addLinkRegex( - Configuration conf, final String mountTableName, - final String srcRegex, final String targetStr) { - addLinkRegex(conf, mountTableName, srcRegex, targetStr, null); - } - /** * Add a LinkRegex to the config for the specified mount table. * @param conf - get mountable config from this conf From 226f8f3358bdcc7742219dc5dbd69e4f1e476d8c Mon Sep 17 00:00:00 2001 From: Zhenzhao Wang Date: Wed, 9 Sep 2020 23:19:45 -0700 Subject: [PATCH 12/12] Address comments. --- .../apache/hadoop/fs/viewfs/InodeTree.java | 95 +++++++++++-------- 1 file changed, 53 insertions(+), 42 deletions(-) diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java index 2af4dc150d9e8..fceb73a56a4fe 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/InodeTree.java @@ -526,11 +526,7 @@ protected InodeTree(final Configuration config, final String viewName, } linkType = LinkType.SINGLE; } else if (src.startsWith(linkFallbackPrefix)) { - if (src.length() != linkFallbackPrefix.length()) { - throw new IOException("ViewFs: Mount points initialization error." + - " Invalid " + Constants.CONFIG_VIEWFS_LINK_FALLBACK + - " entry in config: " + src); - } + checkMntEntryKeyEqualsTarget(src, linkFallbackPrefix); linkType = LinkType.SINGLE_FALLBACK; } else if (src.startsWith(linkMergePrefix)) { // A merge link src = src.substring(linkMergePrefix.length()); @@ -538,11 +534,7 @@ protected InodeTree(final Configuration config, final String viewName, } else if (src.startsWith(linkMergeSlashPrefix)) { // This is a LinkMergeSlash entry. This entry should // not have any additional source path. - if (src.length() != linkMergeSlashPrefix.length()) { - throw new IOException("ViewFs: Mount points initialization error." + - " Invalid " + Constants.CONFIG_VIEWFS_LINK_MERGE_SLASH + - " entry in config: " + src); - } + checkMntEntryKeyEqualsTarget(src, linkMergeSlashPrefix); linkType = LinkType.MERGE_SLASH; } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_NFLY)) { // prefix.settings.src @@ -557,31 +549,8 @@ protected InodeTree(final Configuration config, final String viewName, linkType = LinkType.NFLY; } else if (src.startsWith(Constants.CONFIG_VIEWFS_LINK_REGEX)) { - final String target = si.getValue(); - String linkKeyPath = null; - final String linkRegexPrefix = Constants.CONFIG_VIEWFS_LINK_REGEX + "."; - // settings#.linkKey - String settingsAndLinkKeyPath = src.substring(linkRegexPrefix.length()); - int settingLinkKeySepIndex = settingsAndLinkKeyPath - .indexOf(RegexMountPoint.SETTING_SRCREGEX_SEP); - if (settingLinkKeySepIndex == -1) { - // There's no settings - linkKeyPath = settingsAndLinkKeyPath; - settings = null; - } else { - // settings#.linkKey style configuration - // settings from settings#.linkKey - settings = - settingsAndLinkKeyPath.substring(0, settingLinkKeySepIndex); - // linkKeyPath - linkKeyPath = settingsAndLinkKeyPath.substring( - settings.length() + RegexMountPoint.SETTING_SRCREGEX_SEP - .length()); - } - linkType = LinkType.REGEX; linkEntries.add( - new LinkEntry(linkKeyPath, target, linkType, settings, ugi, - config)); + buildLinkRegexEntry(config, ugi, src, si.getValue())); continue; } else if (src.startsWith(Constants.CONFIG_VIEWFS_HOMEDIR)) { // ignore - we set home dir from config @@ -643,14 +612,7 @@ protected InodeTree(final Configuration config, final String viewName, new URI(le.getTarget())); continue; case REGEX: - LOGGER.info("Add regex mount point:" + le.getSrc() - + ", target:" + le.getTarget() - + ", interceptor settings:" + le.getSettings()); - RegexMountPoint regexMountPoint = - new RegexMountPoint( - this, le.getSrc(), le.getTarget(), le.getSettings()); - regexMountPoint.initialize(); - regexMountPointList.add(regexMountPoint); + addRegexMountEntry(le); continue; default: createLink(le.getSrc(), le.getTarget(), le.getLinkType(), @@ -679,6 +641,55 @@ protected InodeTree(final Configuration config, final String viewName, } } + private void checkMntEntryKeyEqualsTarget( + String mntEntryKey, String targetMntEntryKey) throws IOException { + if (!mntEntryKey.equals(targetMntEntryKey)) { + throw new IOException("ViewFs: Mount points initialization error." + + " Invalid " + targetMntEntryKey + + " entry in config: " + mntEntryKey); + } + } + + private void addRegexMountEntry(LinkEntry le) throws IOException { + LOGGER.info("Add regex mount point:" + le.getSrc() + + ", target:" + le.getTarget() + + ", interceptor settings:" + le.getSettings()); + RegexMountPoint regexMountPoint = + new RegexMountPoint( + this, le.getSrc(), le.getTarget(), le.getSettings()); + regexMountPoint.initialize(); + regexMountPointList.add(regexMountPoint); + } + + private LinkEntry buildLinkRegexEntry( + Configuration config, UserGroupInformation ugi, + String mntEntryStrippedKey, String mntEntryValue) { + String linkKeyPath = null; + String settings = null; + final String linkRegexPrefix = Constants.CONFIG_VIEWFS_LINK_REGEX + "."; + // settings#.linkKey + String settingsAndLinkKeyPath = + mntEntryStrippedKey.substring(linkRegexPrefix.length()); + int settingLinkKeySepIndex = settingsAndLinkKeyPath + .indexOf(RegexMountPoint.SETTING_SRCREGEX_SEP); + if (settingLinkKeySepIndex == -1) { + // There's no settings + linkKeyPath = settingsAndLinkKeyPath; + settings = null; + } else { + // settings#.linkKey style configuration + // settings from settings#.linkKey + settings = + settingsAndLinkKeyPath.substring(0, settingLinkKeySepIndex); + // linkKeyPath + linkKeyPath = settingsAndLinkKeyPath.substring( + settings.length() + RegexMountPoint.SETTING_SRCREGEX_SEP + .length()); + } + return new LinkEntry( + linkKeyPath, mntEntryValue, LinkType.REGEX, settings, ugi, config); + } + /** * Resolve returns ResolveResult. * The caller can continue the resolution of the remainingPath