You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by um...@apache.org on 2020/09/13 03:43:23 UTC

[hadoop] branch branch-3.3 updated: HADOOP-15891. provide Regex Based Mount Point In Inode Tree (#2185). Contributed by Zhenzhao Wang.

This is an automated email from the ASF dual-hosted git repository.

umamahesh pushed a commit to branch branch-3.3
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/branch-3.3 by this push:
     new 2d5ca83  HADOOP-15891. provide Regex Based Mount Point In Inode Tree (#2185). Contributed by Zhenzhao Wang.
2d5ca83 is described below

commit 2d5ca830782016069ce17bd86a1018baa55de148
Author: zz <40...@users.noreply.github.com>
AuthorDate: Thu Sep 10 21:20:32 2020 -0700

    HADOOP-15891. provide Regex Based Mount Point In Inode Tree (#2185). Contributed by Zhenzhao Wang.
    
    Co-authored-by: Zhenzhao Wang <zh...@gmail.com>
    (cherry picked from commit 12a316cdf9994feaa36c3ff7d13e67d70398a9f3)
---
 .../org/apache/hadoop/fs/viewfs/ConfigUtil.java    |  22 +
 .../org/apache/hadoop/fs/viewfs/Constants.java     |   8 +
 .../org/apache/hadoop/fs/viewfs/InodeTree.java     | 340 ++++++++++-----
 .../apache/hadoop/fs/viewfs/RegexMountPoint.java   | 289 +++++++++++++
 .../fs/viewfs/RegexMountPointInterceptor.java      |  70 ++++
 .../viewfs/RegexMountPointInterceptorFactory.java  |  67 +++
 .../fs/viewfs/RegexMountPointInterceptorType.java  |  53 +++
 ...ountPointResolvedDstPathReplaceInterceptor.java | 137 ++++++
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java    |  55 ++-
 .../hadoop/fs/viewfs/TestRegexMountPoint.java      | 160 +++++++
 .../TestRegexMountPointInterceptorFactory.java     |  54 +++
 ...ountPointResolvedDstPathReplaceInterceptor.java | 101 +++++
 .../hadoop-hdfs/src/site/markdown/ViewFs.md        |  63 +++
 .../fs/viewfs/TestViewFileSystemLinkRegex.java     | 462 +++++++++++++++++++++
 14 files changed, 1765 insertions(+), 116 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 7d29b8f..09ec5d2 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
@@ -167,6 +167,28 @@ public class ConfigUtil {
   }
 
   /**
+   * 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
+   * @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,
+      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
    * @param homedir - the home dir path starting with slash
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 492cb87..bf9f7db 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. (^/(?<firstDir>\\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";
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 003694f..dbcd9b4 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
@@ -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;
   }
 
   /**
@@ -453,7 +467,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
@@ -486,84 +500,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);
@@ -578,7 +590,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. "
@@ -588,7 +601,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());
         }
@@ -615,6 +632,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
@@ -647,7 +713,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
@@ -655,13 +721,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;
     }
 
     /**
@@ -677,22 +745,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) {
@@ -709,16 +784,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;
       }
@@ -739,10 +814,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() {
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 0000000..aace7a2
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPoint.java
@@ -0,0 +1,289 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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<T> {
+  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<RegexMountPointInterceptor> 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+))");
+
+  // Same var might have different representations.
+  // e.g.
+  // key => $key or key = > ${key}
+  private Map<String, Set<String>> varInDestPathMap;
+
+  public Map<String, Set<String>> 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 - the string to be process.
+   * @return
+   */
+  public static Map<String, Set<String>> getVarListInString(String input) {
+    Map<String, Set<String>> 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.
+   *  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 - the src path to resolve
+   * @param resolveLastComponent - whether resolve the path after last `/`
+   * @return mapped path of the mount point.
+   */
+  public InodeTree.ResolveResult<T> resolve(final String srcPath,
+      final boolean resolveLastComponent) {
+    String pathStrToResolve = getPathToResolve(srcPath, resolveLastComponent);
+    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<String, Set<String>> varMap = getVarInDestPathMap();
+      for (Map.Entry<String, Set<String>> entry : varMap.entrySet()) {
+        String regexGroupNameOrIndexStr = entry.getKey();
+        Set<String> groupRepresentationStrSetInDest = entry.getValue();
+        parsedDestPath = replaceRegexCaptureGroupInPath(
+            parsedDestPath, srcMatcher,
+            regexGroupNameOrIndexStr, groupRepresentationStrSetInDest);
+      }
+      ++mappedCount;
+    }
+    if (0 == mappedCount) {
+      return null;
+    }
+    Path remainingPath = getRemainingPathStr(srcPath, resolvedPathStr);
+    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;
+  }
+
+  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/(?<username>\\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<String> 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/(?<username>\\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;
+  }
+
+}
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 0000000..37f44b0
--- /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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 0000000..fb564aa
--- /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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 Return interceptor based on setting or null on bad/unknown config.
+   */
+  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 0000000..ad953eb
--- /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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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<String, RegexMountPointInterceptorType>
+      INTERCEPTOR_TYPE_MAP
+      = new HashMap<String, RegexMountPointInterceptorType>();
+
+  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 0000000..18490dc
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/RegexMountPointResolvedDstPathReplaceInterceptor.java
@@ -0,0 +1,137 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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);
+    }
+  }
+
+  /**
+   * Source won't be changed in the interceptor.
+   *
+   * @return source param string passed in.
+   */
+  @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);
+  }
+
+  /**
+   * The interceptRemainingPath will just return the remainingPath passed in.
+   *
+   */
+  @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.
+   *
+   * @return Interceptor instance or null on bad config.
+   */
+  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/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 8c659d1..f981af8 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
@@ -33,7 +33,6 @@ import java.net.URISyntaxException;
 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 +41,7 @@ import java.util.Map;
 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 +114,7 @@ public class ViewFileSystem extends FileSystem {
   static class InnerCache {
     private Map<Key, FileSystem> map = new HashMap<>();
     private FsGetter fsCreator;
+    private ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
 
     InnerCache(FsGetter fsCreator) {
       this.fsCreator = fsCreator;
@@ -121,12 +122,27 @@ public class ViewFileSystem extends FileSystem {
 
     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 +156,13 @@ public class ViewFileSystem extends FileSystem {
       }
     }
 
-    InnerCache unmodifiableCache() {
-      map = Collections.unmodifiableMap(map);
-      return this;
+    void clear() {
+      try {
+        rwLock.writeLock().lock();
+        map.clear();
+      } finally {
+        rwLock.writeLock().unlock();
+      }
     }
 
     /**
@@ -300,13 +320,13 @@ public class ViewFileSystem extends FileSystem {
         @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);
+          FileSystem fs;
+          if (enableInnerCache) {
+            fs = innerCache.get(uri, config);
+          } else {
+            fs = fsGetter.get(uri, config);
+          }
+          return new ChRootedFileSystem(fs, uri);
         }
 
         @Override
@@ -335,7 +355,7 @@ public class ViewFileSystem extends FileSystem {
       // 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;
     }
   }
 
@@ -407,7 +427,7 @@ public class ViewFileSystem extends FileSystem {
       fsState.resolve(getUriPath(f), true);
     return res.targetFileSystem.append(res.remainingPath, bufferSize, progress);
   }
-  
+
   @Override
   public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
       EnumSet<CreateFlag> flags, int bufferSize, short replication,
@@ -1684,6 +1704,7 @@ public class ViewFileSystem extends FileSystem {
     super.close();
     if (enableInnerCache && cache != null) {
       cache.closeAll();
+      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
new file mode 100644
index 0000000..5513b60
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPoint.java
@@ -0,0 +1,160 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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<TestRegexMountPointFileSystem>(conf,
+        TestRegexMountPoint.class.getName(), null, false) {
+      @Override
+      protected TestRegexMountPointFileSystem getTargetFileSystem(
+          final URI uri) {
+        return new TestRegexMountPointFileSystem(uri);
+      }
+
+      @Override
+      protected TestRegexMountPointFileSystem getTargetFileSystem(
+          final INodeDir<TestRegexMountPointFileSystem> 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<String, Set<String>> 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/(?<username>\\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.assertEquals("/user/hadoop", resolveResult.resolvedPath);
+    Assert.assertTrue(
+        resolveResult.targetFileSystem
+            instanceof TestRegexMountPointFileSystem);
+    Assert.assertEquals("/namenode1/testResolve/hadoop",
+        ((TestRegexMountPointFileSystem) resolveResult.targetFileSystem)
+            .getUri().toString());
+    Assert.assertEquals("/file1", resolveResult.remainingPath.toString());
+  }
+
+  @Test
+  public void testResolveWithInterceptor() throws IOException {
+    String regexStr = "^/user/(?<username>\\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.assertEquals("/user/hadoop_user1", resolveResult.resolvedPath);
+    Assert.assertTrue(
+        resolveResult.targetFileSystem
+            instanceof TestRegexMountPointFileSystem);
+    Assert.assertEquals("/namenode1/testResolve/hadoop-user1",
+        ((TestRegexMountPointFileSystem) resolveResult.targetFileSystem)
+            .getUri().toString());
+    Assert.assertEquals("/file_index",
+        resolveResult.remainingPath.toString());
+  }
+}
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 0000000..c567944
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointInterceptorFactory.java
@@ -0,0 +1,54 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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 0000000..9fdf0f6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestRegexMountPointResolvedDstPathReplaceInterceptor.java
@@ -0,0 +1,101 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.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.assertEquals(srcRegex, interceptor.getSrcRegexString());
+    Assert.assertEquals(replaceString, interceptor.getReplaceString());
+    Assert.assertNull(interceptor.getSrcRegexPattern());
+    interceptor.initialize();
+    Assert.assertEquals(srcRegex,
+        interceptor.getSrcRegexPattern().toString());
+  }
+
+  @Test
+  public void testDeserializeFromStringBadCase() throws IOException {
+    String srcRegex = "-";
+    String replaceString = "_";
+    String serializedString = createSerializedString(srcRegex, replaceString);
+    serializedString = serializedString + ":ddd";
+    RegexMountPointResolvedDstPathReplaceInterceptor interceptor =
+        RegexMountPointResolvedDstPathReplaceInterceptor
+            .deserializeFromString(serializedString);
+    Assert.assertNull(interceptor);
+  }
+
+  @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";
+    Assert.assertEquals(sourcePath, interceptor.interceptSource(sourcePath));
+  }
+
+  @Test
+  public void testInterceptResolve() throws IOException {
+    String pathAfterResolution = "/user-hadoop";
+
+    String srcRegex = "hadoop";
+    String replaceString = "hdfs";
+    RegexMountPointResolvedDstPathReplaceInterceptor interceptor =
+        new RegexMountPointResolvedDstPathReplaceInterceptor(srcRegex,
+            replaceString);
+    interceptor.initialize();
+    Assert.assertEquals("/user-hdfs",
+        interceptor.interceptResolvedDestPathStr(pathAfterResolution));
+  }
+}
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 52ad49c..b29a888 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,69 @@ 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 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
+<property>
+    <name>fs.viewfs.rename.strategy</name>
+    <value>SAME_FILESYSTEM_ACROSS_MOUNTPOINT</value>
+</property>
+```
+
+### 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
+<property>
+    <name>fs.viewfs.mounttable.hadoop-nn.linkRegx./^(?<username>\\w+)</name>
+    <value>gs://${username}.hadoop.apache.org/</value>
+</property>
+```
+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
+<property>
+    <name>fs.viewfs.mounttable.hadoop-nn.linkRegx.replaceresolveddstpath:_:-#./^(?<username>\\w+)</name>
+    <value>gs://${username}.hadoop.apache.org/</value>
+</property>
+```
+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}
+```
+
+
 
 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 0000000..d3afa47
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemLinkRegex.java
@@ -0,0 +1,462 @@
+/**
+ * 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
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * 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.net.URISyntaxException;
+import java.util.ArrayList;
+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.RegexMountPoint.INTERCEPTOR_INTERNAL_SEP;
+import static org.junit.Assert.assertSame;
+
+/**
+ * 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<String> 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();
+  }
+
+  private void createDirWithChildren(
+      FileSystem fileSystem, Path dir, List<Path> 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<Path> 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 {
+      // 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());
+    }
+  }
+  /**
+   * Test regex mount points which use capture group index for mapping.
+   *
+   * @throws Exception
+   */
+  @Test
+  public void testConfLinkRegexIndexMapping() throws Exception {
+    //  Config:
+    //   <property>
+    //     <name>
+    //     fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.^/(\w+)</name>
+    //     <value>/targetTestRoot/$1</value>
+    //   </property>
+    // Dir path to test: /testConfLinkRegexIndexMapping1
+    // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping1
+    String regexStr = "^/(\\w+)";
+    String dstPathStr = targetTestRoot + "$1";
+    Path srcPath = new Path("/testConfLinkRegexIndexMapping1");
+    Path expectedResolveResult = new Path(dstPathStr.replace(
+          "$1", "testConfLinkRegexIndexMapping1"));
+    testRegexMountpoint(
+        new Configuration(conf),
+        regexStr, dstPathStr, null,
+        srcPath, expectedResolveResult, 3);
+
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.^/(\w+)</name>
+    //     <value>/targetTestRoot/${1}</value>
+    //   </property>
+    // Dir path to test: /testConfLinkRegexIndexMapping2
+    // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping2
+
+    dstPathStr = targetTestRoot + "${1}";
+    srcPath = new Path("/testConfLinkRegexIndexMapping2");
+    expectedResolveResult =
+        new Path(
+            dstPathStr.replace("${1}", "testConfLinkRegexIndexMapping2"));
+    testRegexMountpoint(
+        new Configuration(conf),
+        regexStr, dstPathStr, null,
+        srcPath, expectedResolveResult, 4);
+
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.^/(\w+)</name>
+    //     <value>/targetTestRoot/$1</value>
+    //   </property>
+    // Dir path to test: /testConfLinkRegexIndexMapping3/dir1
+    // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping3/dir1
+    dstPathStr = targetTestRoot + "$1";
+    srcPath = new Path("/testConfLinkRegexIndexMapping3/dir1");
+    expectedResolveResult = new Path(
+        dstPathStr.replace("$1", "testConfLinkRegexIndexMapping3/dir1"));
+    testRegexMountpoint(
+        new Configuration(conf),
+        regexStr, dstPathStr, null,
+        srcPath, expectedResolveResult, 5);
+
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.^/(\w+)</name>
+    //     <value>/targetTestRoot/${1}/</value>
+    //   </property>
+    // Dir path to test: /testConfLinkRegexIndexMapping4/dir1
+    // Expect path: /targetTestRoot/testConfLinkRegexIndexMapping4/dir1
+    dstPathStr = targetTestRoot + "${1}/";
+    srcPath = new Path("/testConfLinkRegexIndexMapping4/dir1");
+    expectedResolveResult = new Path(
+          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 {
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.^/(?<firstDir>\w+)</name>
+    //     <value>/targetTestRoot/$firstDir</value>
+    //   </property>
+    // Dir path to test: /testConfLinkRegexNamedGroupMapping1
+    // Expect path: /targetTestRoot/testConfLinkRegexNamedGroupMapping1
+    URI viewFsUri = new URI(
+        FsConstants.VIEWFS_SCHEME, CLUSTER_NAME, "/", null, null);
+    String regexStr = "^/(?<firstDir>\\w+)";
+    String dstPathStr = targetTestRoot + "$firstDir";
+    Path srcPath = new Path("/testConfLinkRegexNamedGroupMapping1");
+    Path expectedResolveResult = new Path(
+        dstPathStr.replace("$firstDir", "testConfLinkRegexNamedGroupMapping1"));
+    testRegexMountpoint(
+        new Configuration(conf),
+        regexStr, dstPathStr, null,
+        srcPath, expectedResolveResult, 3);
+
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.^/(?<firstDir>\w+)</name>
+    //     <value>/targetTestRoot/${firstDir}</value>
+    //   </property>
+    // Dir path to test: /testConfLinkRegexNamedGroupMapping2
+    // Expect path: /targetTestRoot/testConfLinkRegexNamedGroupMapping2
+    dstPathStr = targetTestRoot + "${firstDir}";
+    srcPath = new Path("/testConfLinkRegexNamedGroupMapping2");
+    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 {
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.^/(?<firstDir>\w+)</name>
+    //     <value>/targetTestRoot/${firstDir}</value>
+    //   </property>
+    // Dir path to test: /misc1
+    // Expect path: /targetTestRoot/testConfLinkRegexFixedDestMappingFile
+    // Dir path to test: /misc2
+    // Expect path: /targetTestRoot/testConfLinkRegexFixedDestMappingFile
+    String regexStr = "^/\\w+";
+    String dstPathStr =
+        targetTestRoot + "testConfLinkRegexFixedDestMappingFile";
+    Path expectedResolveResult = new Path(dstPathStr);
+    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 {
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex.replaceresolveddstpath:_:-#.^/user/(?<username>\w+)</name>
+    //     <value>/targetTestRoot/$username</value>
+    //   </property>
+    // Dir path to test: /user/hadoop_user1/hadoop_dir1
+    // Expect path: /targetTestRoot/hadoop-user1/hadoop_dir1
+
+    String regexStr = "^/user/(?<username>\\w+)";
+    String dstPathStr = targetTestRoot + "$username";
+    // Replace "_" with "-"
+    String settingString = buildReplaceInterceptorSettingString("_", "-");
+    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 {
+    // Config:
+    //   <property>
+    //     <name>fs.viewfs.mounttable.TestViewFileSystemLinkRegexCluster
+    //     .linkRegex
+    //     .replaceresolveddstpath:_:-;
+    //     replaceresolveddstpath:hadoop:hdfs#.^/user/(?<username>\w+)</name>
+    //     <value>/targetTestRoot/$username</value>
+    //   </property>
+    // Dir path to test: /user/hadoop_user1/hadoop_dir1
+    // Expect path: /targetTestRoot/hdfs-user1/hadoop_dir1
+    String regexStr = "^/user/(?<username>\\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_dir1");
+    Path expectedResolveResult =
+        new Path(targetTestRoot, "hdfs-user1/hadoop_dir1");
+    testRegexMountpoint(
+        new Configuration(conf),
+        regexStr, dstPathStr, interceptors,
+        srcPath, expectedResolveResult, 2);
+
+  }
+}


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