|
@@ -39,6 +39,8 @@ import org.apache.hadoop.fs.Path;
|
|
|
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 +48,22 @@ import org.apache.hadoop.util.StringUtils;
|
|
|
* 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 <T> 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<T> {
|
|
|
+ private static final Logger LOGGER =
|
|
|
+ LoggerFactory.getLogger(InodeTree.class.getName());
|
|
|
+
|
|
|
enum ResultKind {
|
|
|
INTERNAL_DIR,
|
|
|
EXTERNAL_DIR
|
|
@@ -72,6 +77,8 @@ abstract class InodeTree<T> {
|
|
|
// the homedir for this mount table
|
|
|
private final String homedirPrefix;
|
|
|
private List<MountPoint<T>> mountPoints = new ArrayList<MountPoint<T>>();
|
|
|
+ private List<RegexMountPoint<T>> regexMountPointList =
|
|
|
+ new ArrayList<RegexMountPoint<T>>();
|
|
|
|
|
|
static class MountPoint<T> {
|
|
|
String src;
|
|
@@ -122,7 +129,7 @@ abstract class InodeTree<T> {
|
|
|
*/
|
|
|
static class INodeDir<T> extends INode<T> {
|
|
|
private final Map<String, INode<T>> 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<T> fallbackLink = null;
|
|
|
|
|
@@ -226,7 +233,14 @@ abstract class InodeTree<T> {
|
|
|
* Config prefix: fs.viewfs.mounttable.<mnt_tbl_name>.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.<mnt_tbl_name>.linkRegex
|
|
|
+ * Refer: {@link Constants#CONFIG_VIEWFS_LINK_REGEX}
|
|
|
+ */
|
|
|
+ REGEX;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -462,7 +476,7 @@ abstract class InodeTree<T> {
|
|
|
|
|
|
/**
|
|
|
* 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 +509,82 @@ abstract class InodeTree<T> {
|
|
|
final UserGroupInformation ugi = UserGroupInformation.getCurrentUser();
|
|
|
for (Entry<String, String> 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)) {
|
|
|
+ checkMntEntryKeyEqualsTarget(src, linkFallbackPrefix);
|
|
|
+ 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.
|
|
|
+ checkMntEntryKeyEqualsTarget(src, linkMergeSlashPrefix);
|
|
|
+ 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)) {
|
|
|
+ linkEntries.add(
|
|
|
+ buildLinkRegexEntry(config, ugi, src, si.getValue()));
|
|
|
+ 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;
|
|
|
+ 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 +599,8 @@ abstract class InodeTree<T> {
|
|
|
getRootDir().setRoot(true);
|
|
|
INodeLink<T> 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 +610,11 @@ abstract class InodeTree<T> {
|
|
|
fallbackLink = new INodeLink<T>(mountTableName, ugi,
|
|
|
getTargetFileSystem(new URI(le.getTarget())),
|
|
|
new URI(le.getTarget()));
|
|
|
- } else {
|
|
|
+ continue;
|
|
|
+ case REGEX:
|
|
|
+ addRegexMountEntry(le);
|
|
|
+ continue;
|
|
|
+ default:
|
|
|
createLink(le.getSrc(), le.getTarget(), le.getLinkType(),
|
|
|
le.getSettings(), le.getUgi(), le.getConfig());
|
|
|
}
|
|
@@ -624,6 +641,55 @@ abstract class InodeTree<T> {
|
|
|
}
|
|
|
}
|
|
|
|
|
|
+ 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<T>(
|
|
|
+ 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
|
|
@@ -656,7 +722,7 @@ abstract class InodeTree<T> {
|
|
|
}
|
|
|
|
|
|
/**
|
|
|
- * 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,13 +730,15 @@ abstract class InodeTree<T> {
|
|
|
*/
|
|
|
ResolveResult<T> resolve(final String p, final boolean resolveLastComponent)
|
|
|
throws FileNotFoundException {
|
|
|
+ ResolveResult<T> 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<T> res = new ResolveResult<T>(ResultKind.INTERNAL_DIR,
|
|
|
+ getRootDir().getInternalDirFs()
|
|
|
+ : getRootLink().getTargetFileSystem();
|
|
|
+ resolveResult = new ResolveResult<T>(ResultKind.INTERNAL_DIR,
|
|
|
targetFs, root.fullPath, SlashPath);
|
|
|
- return res;
|
|
|
+ return resolveResult;
|
|
|
}
|
|
|
|
|
|
/**
|
|
@@ -686,22 +754,29 @@ abstract class InodeTree<T> {
|
|
|
remainingPathStr.append("/").append(path[i]);
|
|
|
}
|
|
|
remainingPath = new Path(remainingPathStr.toString());
|
|
|
- ResolveResult<T> res = new ResolveResult<T>(ResultKind.EXTERNAL_DIR,
|
|
|
+ resolveResult = new ResolveResult<T>(ResultKind.EXTERNAL_DIR,
|
|
|
getRootLink().getTargetFileSystem(), root.fullPath, remainingPath);
|
|
|
- return res;
|
|
|
+ return resolveResult;
|
|
|
}
|
|
|
Preconditions.checkState(root.isInternalDir());
|
|
|
INodeDir<T> curInode = getRootDir();
|
|
|
|
|
|
+ // 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<T> nextInode = curInode.resolveInternal(path[i]);
|
|
|
if (nextInode == null) {
|
|
|
if (hasFallbackLink()) {
|
|
|
- return new ResolveResult<T>(ResultKind.EXTERNAL_DIR,
|
|
|
- getRootFallbackLink().getTargetFileSystem(),
|
|
|
- root.fullPath, new Path(p));
|
|
|
+ resolveResult = new ResolveResult<T>(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) {
|
|
@@ -718,16 +793,16 @@ abstract class InodeTree<T> {
|
|
|
if (i >= path.length - 1) {
|
|
|
remainingPath = SlashPath;
|
|
|
} else {
|
|
|
- StringBuilder remainingPathStr = new StringBuilder("/" + path[i + 1]);
|
|
|
+ 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());
|
|
|
}
|
|
|
- final ResolveResult<T> res =
|
|
|
- new ResolveResult<T>(ResultKind.EXTERNAL_DIR,
|
|
|
- link.getTargetFileSystem(), nextInode.fullPath, remainingPath);
|
|
|
- return res;
|
|
|
+ resolveResult = new ResolveResult<T>(ResultKind.EXTERNAL_DIR,
|
|
|
+ link.getTargetFileSystem(), nextInode.fullPath, remainingPath);
|
|
|
+ return resolveResult;
|
|
|
} else if (nextInode.isInternalDir()) {
|
|
|
curInode = (INodeDir<T>) nextInode;
|
|
|
}
|
|
@@ -748,10 +823,77 @@ abstract class InodeTree<T> {
|
|
|
}
|
|
|
remainingPath = new Path(remainingPathStr.toString());
|
|
|
}
|
|
|
- final ResolveResult<T> res =
|
|
|
- new ResolveResult<T>(ResultKind.INTERNAL_DIR,
|
|
|
- curInode.getInternalDirFs(), curInode.fullPath, remainingPath);
|
|
|
- return res;
|
|
|
+ resolveResult = new ResolveResult<T>(ResultKind.INTERNAL_DIR,
|
|
|
+ curInode.getInternalDirFs(), curInode.fullPath, remainingPath);
|
|
|
+ return resolveResult;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Walk through all regex mount points to see
|
|
|
+ * whether the path match any regex expressions.
|
|
|
+ * E.g. link: ^/user/(?<username>\\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
|
|
|
+ * @return
|
|
|
+ */
|
|
|
+ protected ResolveResult<T> tryResolveInRegexMountpoint(final String srcPath,
|
|
|
+ final boolean resolveLastComponent) {
|
|
|
+ for (RegexMountPoint regexMountPoint : regexMountPointList) {
|
|
|
+ ResolveResult resolveResult =
|
|
|
+ regexMountPoint.resolve(srcPath, resolveLastComponent);
|
|
|
+ if (resolveResult != null) {
|
|
|
+ return resolveResult;
|
|
|
+ }
|
|
|
+ }
|
|
|
+ return null;
|
|
|
+ }
|
|
|
+
|
|
|
+ /**
|
|
|
+ * Build resolve result.
|
|
|
+ * Here's an example
|
|
|
+ * Mountpoint: fs.viewfs.mounttable.mt
|
|
|
+ * .linkRegex.replaceresolveddstpath:_:-#.^/user/(?<username>\w+)
|
|
|
+ * Value: /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.
|
|
|
+ */
|
|
|
+ protected ResolveResult<T> buildResolveResultForRegexMountPoint(
|
|
|
+ ResultKind resultKind, String resolvedPathStr,
|
|
|
+ String targetOfResolvedPathStr, Path remainingPath) {
|
|
|
+ try {
|
|
|
+ T targetFs = getTargetFileSystem(
|
|
|
+ new URI(targetOfResolvedPathStr));
|
|
|
+ return new ResolveResult<T>(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;
|
|
|
+ }
|
|
|
}
|
|
|
|
|
|
List<MountPoint<T>> getMountPoints() {
|