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/06/27 22:42:09 UTC

[hadoop] branch branch-3.3 updated: HDFS-15418. ViewFileSystemOverloadScheme should represent mount links as non symlinks. Contributed by Uma Maheswara Rao G.

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 5f67c3f  HDFS-15418. ViewFileSystemOverloadScheme should represent mount links as non symlinks. Contributed by Uma Maheswara Rao G.
5f67c3f is described below

commit 5f67c3f3ca49d718d8fc7c1914c3d2b77b3462f0
Author: Uma Maheswara Rao G <um...@apache.org>
AuthorDate: Sat Jun 20 00:32:02 2020 -0700

    HDFS-15418. ViewFileSystemOverloadScheme should represent mount links as non symlinks. Contributed by Uma Maheswara Rao G.
    
    (cherry picked from commit b27810aa6015253866ccc0ccc7247ad7024c0730)
---
 .../org/apache/hadoop/fs/viewfs/Constants.java     |   8 ++
 .../apache/hadoop/fs/viewfs/ViewFileSystem.java    |  71 +++++++----
 .../fs/viewfs/ViewFileSystemOverloadScheme.java    |  20 +++-
 .../java/org/apache/hadoop/fs/viewfs/ViewFs.java   |  80 ++++++++-----
 .../viewfs/TestViewFsOverloadSchemeListStatus.java | 132 +++++++++++++++++++++
 .../hadoop/fs/viewfs/TestViewfsFileStatus.java     |   4 +-
 .../src/site/markdown/ViewFsOverloadScheme.md      |  42 +++----
 ...SystemOverloadSchemeHdfsFileSystemContract.java |   5 +
 ...ViewFileSystemOverloadSchemeWithHdfsScheme.java |   9 ++
 9 files changed, 295 insertions(+), 76 deletions(-)

diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/Constants.java
index 0a5d4b4..f454f63 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
@@ -90,4 +90,12 @@ public interface Constants {
   String CONFIG_VIEWFS_ENABLE_INNER_CACHE = "fs.viewfs.enable.inner.cache";
 
   boolean CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT = true;
+
+  /**
+   * Enable ViewFileSystem to show mountlinks as symlinks.
+   */
+  String CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS =
+      "fs.viewfs.mount.links.as.symlinks";
+
+  boolean CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT = true;
 }
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 895edc0..1ee06e0 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
@@ -20,6 +20,8 @@ package org.apache.hadoop.fs.viewfs;
 import static org.apache.hadoop.fs.impl.PathCapabilitiesSupport.validatePathCapabilityArgs;
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE;
 import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_ENABLE_INNER_CACHE_DEFAULT;
+import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS;
+import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT;
 import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_555;
 
 import java.io.FileNotFoundException;
@@ -527,10 +529,18 @@ public class ViewFileSystem extends FileSystem {
    * the target path FileStatus object. The target path will be available via
    * getSymlink on that children's FileStatus object. Since it represents as
    * symlink, isDirectory on that children's FileStatus will return false.
+   * This behavior can be changed by setting an advanced configuration
+   * fs.viewfs.mount.links.as.symlinks to false. In this case, mount points will
+   * be represented as non-symlinks and all the file/directory attributes like
+   * permissions, isDirectory etc will be assigned from it's resolved target
+   * directory/file.
    *
    * If you want to get the FileStatus of target path for that children, you may
    * want to use GetFileStatus API with that children's symlink path. Please see
    * {@link ViewFileSystem#getFileStatus(Path f)}
+   *
+   * Note: In ViewFileSystem, by default the mount links are represented as
+   * symlinks.
    */
   @Override
   public FileStatus[] listStatus(final Path f) throws AccessControlException,
@@ -1114,6 +1124,7 @@ public class ViewFileSystem extends FileSystem {
     final long creationTime; // of the the mount table
     final UserGroupInformation ugi; // the user/group of user who created mtable
     final URI myUri;
+    private final boolean showMountLinksAsSymlinks;
     
     public InternalDirOfViewFs(final InodeTree.INodeDir<FileSystem> dir,
         final long cTime, final UserGroupInformation ugi, URI uri,
@@ -1127,6 +1138,9 @@ public class ViewFileSystem extends FileSystem {
       theInternalDir = dir;
       creationTime = cTime;
       this.ugi = ugi;
+      showMountLinksAsSymlinks = config
+          .getBoolean(CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS,
+              CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT);
     }
 
     static private void checkPathIsSlash(final Path f) throws IOException {
@@ -1216,37 +1230,50 @@ public class ViewFileSystem extends FileSystem {
       for (Entry<String, INode<FileSystem>> iEntry :
           theInternalDir.getChildren().entrySet()) {
         INode<FileSystem> inode = iEntry.getValue();
+        Path path = new Path(inode.fullPath).makeQualified(myUri, null);
         if (inode.isLink()) {
           INodeLink<FileSystem> link = (INodeLink<FileSystem>) inode;
+
+          if (showMountLinksAsSymlinks) {
+            // To maintain backward compatibility, with default option(showing
+            // mount links as symlinks), we will represent target link as
+            // symlink and rest other properties are belongs to mount link only.
+            result[i++] =
+                new FileStatus(0, false, 0, 0, creationTime, creationTime,
+                    PERMISSION_555, ugi.getShortUserName(),
+                    ugi.getPrimaryGroupName(), link.getTargetLink(),
+                    path);
+            continue;
+          }
+
+          //  We will represent as non-symlinks. Here it will show target
+          //  directory/file properties like permissions, isDirectory etc on
+          //  mount path. The path will be a mount link path and isDirectory is
+          //  true if target is dir, otherwise false.
+          String linkedPath = link.getTargetFileSystem().getUri().getPath();
+          if ("".equals(linkedPath)) {
+            linkedPath = "/";
+          }
           try {
-            String linkedPath = link.getTargetFileSystem().getUri().getPath();
-            if("".equals(linkedPath)) {
-              linkedPath = "/";
-            }
             FileStatus status =
                 ((ChRootedFileSystem)link.getTargetFileSystem())
                 .getMyFs().getFileStatus(new Path(linkedPath));
-            result[i++] = new FileStatus(status.getLen(), false,
-              status.getReplication(), status.getBlockSize(),
-              status.getModificationTime(), status.getAccessTime(),
-              status.getPermission(), status.getOwner(), status.getGroup(),
-              link.getTargetLink(),
-              new Path(inode.fullPath).makeQualified(
-                  myUri, null));
+            result[i++] = new FileStatus(status.getLen(), status.isDirectory(),
+                status.getReplication(), status.getBlockSize(),
+                status.getModificationTime(), status.getAccessTime(),
+                status.getPermission(), status.getOwner(), status.getGroup(),
+                null, path);
           } catch (FileNotFoundException ex) {
-            result[i++] = new FileStatus(0, false, 0, 0,
-              creationTime, creationTime, PERMISSION_555,
-              ugi.getShortUserName(), ugi.getPrimaryGroupName(),
-              link.getTargetLink(),
-              new Path(inode.fullPath).makeQualified(
-                  myUri, null));
+            LOG.warn("Cannot get one of the children's(" + path
+                + ")  target path(" + link.getTargetFileSystem().getUri()
+                + ") file status.", ex);
+            throw ex;
           }
         } else {
-          result[i++] = new FileStatus(0, true, 0, 0,
-            creationTime, creationTime, PERMISSION_555,
-            ugi.getShortUserName(), ugi.getPrimaryGroupName(),
-            new Path(inode.fullPath).makeQualified(
-                myUri, null));
+          result[i++] =
+              new FileStatus(0, true, 0, 0, creationTime, creationTime,
+                  PERMISSION_555, ugi.getShortUserName(),
+                  ugi.getPrimaryGroupName(), path);
         }
       }
       if (fallbackStatuses.length > 0) {
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java
index 36f9cd1..672022b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemOverloadScheme.java
@@ -59,9 +59,9 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
  * data to mount with other hdfs and object store clusters(hdfs://NN1,
  * o3fs://bucket1.volume1/, s3a://bucket1/)
  *
- * fs.viewfs.mounttable.Cluster./user = hdfs://NN1/user
- * fs.viewfs.mounttable.Cluster./data = o3fs://bucket1.volume1/data
- * fs.viewfs.mounttable.Cluster./backup = s3a://bucket1/backup/
+ * fs.viewfs.mounttable.Cluster.link./user = hdfs://NN1/user
+ * fs.viewfs.mounttable.Cluster.link./data = o3fs://bucket1.volume1/data
+ * fs.viewfs.mounttable.Cluster.link./backup = s3a://bucket1/backup/
  *
  * Op1: Create file hdfs://Cluster/user/fileA will go to hdfs://NN1/user/fileA
  * Op2: Create file hdfs://Cluster/data/datafile will go to
@@ -75,15 +75,19 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
  * data to mount with other hdfs and object store clusters
  * (hdfs://NN1, o3fs://bucket1.volume1/)
  *
- * fs.viewfs.mounttable.bucketA./user = hdfs://NN1/user
- * fs.viewfs.mounttable.bucketA./data = o3fs://bucket1.volume1/data
- * fs.viewfs.mounttable.bucketA./salesDB = s3a://bucketA/salesDB/
+ * fs.viewfs.mounttable.bucketA.link./user = hdfs://NN1/user
+ * fs.viewfs.mounttable.bucketA.link./data = o3fs://bucket1.volume1/data
+ * fs.viewfs.mounttable.bucketA.link./salesDB = s3a://bucketA/salesDB/
  *
  * Op1: Create file s3a://bucketA/user/fileA will go to hdfs://NN1/user/fileA
  * Op2: Create file s3a://bucketA/data/datafile will go to
  *      o3fs://bucket1.volume1/data/datafile
  * Op3: Create file s3a://bucketA/salesDB/dbfile will go to
  *      s3a://bucketA/salesDB/dbfile
+ *
+ * Note: In ViewFileSystemOverloadScheme, by default the mount links will be
+ * represented as non-symlinks. If you want to change this behavior, please see
+ * {@link ViewFileSystem#listStatus(Path)}
  *****************************************************************************/
 @InterfaceAudience.LimitedPrivate({ "MapReduce", "HBase", "Hive" })
 @InterfaceStability.Evolving
@@ -107,6 +111,10 @@ public class ViewFileSystemOverloadScheme extends ViewFileSystem {
     }
     String mountTableConfigPath =
         conf.get(Constants.CONFIG_VIEWFS_MOUNTTABLE_PATH);
+    /* The default value to false in ViewFSOverloadScheme */
+    conf.setBoolean(Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS,
+        conf.getBoolean(Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS,
+            false));
     if (null != mountTableConfigPath) {
       MountTableConfigLoader loader = new HCFSMountTableConfigLoader();
       loader.load(mountTableConfigPath, conf);
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
index 4578a4c..fae5d1b 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFs.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.fs.viewfs;
 
+import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS;
+import static org.apache.hadoop.fs.viewfs.Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT;
 import static org.apache.hadoop.fs.viewfs.Constants.PERMISSION_555;
 
 import java.io.FileNotFoundException;
@@ -67,7 +69,8 @@ import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.Progressable;
 import org.apache.hadoop.util.Time;
-
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 /**
  * ViewFs (extends the AbstractFileSystem interface) implements a client-side
@@ -154,6 +157,7 @@ import org.apache.hadoop.util.Time;
 @InterfaceAudience.Public
 @InterfaceStability.Evolving /*Evolving for a release,to be changed to Stable */
 public class ViewFs extends AbstractFileSystem {
+  static final Logger LOG = LoggerFactory.getLogger(ViewFs.class);
   final long creationTime; // of the the mount table
   final UserGroupInformation ugi; // the user/group of user who created mtable
   final Configuration config;
@@ -161,6 +165,7 @@ public class ViewFs extends AbstractFileSystem {
   Path homeDir = null;
   private ViewFileSystem.RenameStrategy renameStrategy =
       ViewFileSystem.RenameStrategy.SAME_MOUNTPOINT;
+  private static boolean showMountLinksAsSymlinks = true;
 
   static AccessControlException readOnlyMountTable(final String operation,
       final String p) {
@@ -209,6 +214,9 @@ public class ViewFs extends AbstractFileSystem {
     creationTime = Time.now();
     ugi = UserGroupInformation.getCurrentUser();
     config = conf;
+    showMountLinksAsSymlinks = config
+        .getBoolean(CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS,
+            CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS_DEFAULT);
     // Now build  client side view (i.e. client side mount table) from config.
     String authority = theUri.getAuthority();
     fsState = new InodeTree<AbstractFileSystem>(conf, authority) {
@@ -453,12 +461,17 @@ public class ViewFs extends AbstractFileSystem {
    * the target path FileStatus object. The target path will be available via
    * getSymlink on that children's FileStatus object. Since it represents as
    * symlink, isDirectory on that children's FileStatus will return false.
+   * This behavior can be changed by setting an advanced configuration
+   * fs.viewfs.mount.links.as.symlinks to false. In this case, mount points will
+   * be represented as non-symlinks and all the file/directory attributes like
+   * permissions, isDirectory etc will be assigned from it's resolved target
+   * directory/file.
    *
    * If you want to get the FileStatus of target path for that children, you may
    * want to use GetFileStatus API with that children's symlink path. Please see
    * {@link ViewFs#getFileStatus(Path f)}
    *
-   * Note: In ViewFs, the mount links are represented as symlinks.
+   * Note: In ViewFs, by default the mount links are represented as symlinks.
    */
   @Override
   public FileStatus[] listStatus(final Path f) throws AccessControlException,
@@ -999,8 +1012,7 @@ public class ViewFs extends AbstractFileSystem {
      * will be listed in the returned result.
      */
     @Override
-    public FileStatus[] listStatus(final Path f) throws AccessControlException,
-        IOException {
+    public FileStatus[] listStatus(final Path f) throws IOException {
       checkPathIsSlash(f);
       FileStatus[] fallbackStatuses = listStatusForFallbackLink();
       FileStatus[] result = new FileStatus[theInternalDir.getChildren().size()];
@@ -1008,37 +1020,51 @@ public class ViewFs extends AbstractFileSystem {
       for (Entry<String, INode<AbstractFileSystem>> iEntry :
           theInternalDir.getChildren().entrySet()) {
         INode<AbstractFileSystem> inode = iEntry.getValue();
-
-        
+        Path path = new Path(inode.fullPath).makeQualified(myUri, null);
         if (inode.isLink()) {
           INodeLink<AbstractFileSystem> link = 
             (INodeLink<AbstractFileSystem>) inode;
 
+          if (showMountLinksAsSymlinks) {
+            // To maintain backward compatibility, with default option(showing
+            // mount links as symlinks), we will represent target link as
+            // symlink and rest other properties are belongs to mount link only.
+            result[i++] =
+                new FileStatus(0, false, 0, 0, creationTime, creationTime,
+                    PERMISSION_555, ugi.getShortUserName(),
+                    ugi.getPrimaryGroupName(), link.getTargetLink(),
+                    path);
+            continue;
+          }
+
+          //  We will represent as non-symlinks. Here it will show target
+          //  directory/file properties like permissions, isDirectory etc on
+          //  mount path. The path will be a mount link path and isDirectory is
+          //  true if target is dir, otherwise false.
+          String linkedPath = link.getTargetFileSystem().getUri().getPath();
+          if ("".equals(linkedPath)) {
+            linkedPath = "/";
+          }
           try {
-            String linkedPath = link.getTargetFileSystem().getUri().getPath();
-            FileStatus status = ((ChRootedFs)link.getTargetFileSystem())
-                .getMyFs().getFileStatus(new Path(linkedPath));
-            result[i++] = new FileStatus(status.getLen(), false,
-              status.getReplication(), status.getBlockSize(),
-              status.getModificationTime(), status.getAccessTime(),
-              status.getPermission(), status.getOwner(), status.getGroup(),
-              link.getTargetLink(),
-              new Path(inode.fullPath).makeQualified(
-                  myUri, null));
+            FileStatus status =
+                ((ChRootedFs) link.getTargetFileSystem()).getMyFs()
+                    .getFileStatus(new Path(linkedPath));
+            result[i++] = new FileStatus(status.getLen(), status.isDirectory(),
+                status.getReplication(), status.getBlockSize(),
+                status.getModificationTime(), status.getAccessTime(),
+                status.getPermission(), status.getOwner(), status.getGroup(),
+                null, path);
           } catch (FileNotFoundException ex) {
-            result[i++] = new FileStatus(0, false, 0, 0,
-              creationTime, creationTime, PERMISSION_555,
-              ugi.getShortUserName(), ugi.getPrimaryGroupName(),
-              link.getTargetLink(),
-              new Path(inode.fullPath).makeQualified(
-                  myUri, null));
+            LOG.warn("Cannot get one of the children's(" + path
+                + ")  target path(" + link.getTargetFileSystem().getUri()
+                + ") file status.", ex);
+            throw ex;
           }
         } else {
-          result[i++] = new FileStatus(0, true, 0, 0,
-            creationTime, creationTime,
-            PERMISSION_555, ugi.getShortUserName(), ugi.getGroupNames()[0],
-            new Path(inode.fullPath).makeQualified(
-                myUri, null));
+          result[i++] =
+              new FileStatus(0, true, 0, 0, creationTime, creationTime,
+                  PERMISSION_555, ugi.getShortUserName(),
+                  ugi.getGroupNames()[0], path);
         }
       }
       if (fallbackStatuses.length > 0) {
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsOverloadSchemeListStatus.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsOverloadSchemeListStatus.java
new file mode 100644
index 0000000..0cf6914
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFsOverloadSchemeListStatus.java
@@ -0,0 +1,132 @@
+/**
+ * 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.FileOutputStream;
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.LocalFileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.test.GenericTestUtils;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * ViewFsOverloadScheme ListStatus.
+ */
+public class TestViewFsOverloadSchemeListStatus {
+
+  private static final File TEST_DIR =
+      GenericTestUtils.getTestDir(TestViewfsFileStatus.class.getSimpleName());
+
+  @Before
+  public void setUp() {
+    FileUtil.fullyDelete(TEST_DIR);
+    assertTrue(TEST_DIR.mkdirs());
+  }
+
+  @After
+  public void tearDown() throws IOException {
+    FileUtil.fullyDelete(TEST_DIR);
+  }
+
+  /**
+   * Tests the ACL and isDirectory returned from listStatus for directories and
+   * files.
+   */
+  @Test
+  public void testListStatusACL() throws IOException, URISyntaxException {
+    String testfilename = "testFileACL";
+    String childDirectoryName = "testDirectoryACL";
+    TEST_DIR.mkdirs();
+    File infile = new File(TEST_DIR, testfilename);
+    final byte[] content = "dingos".getBytes();
+
+    try (FileOutputStream fos = new FileOutputStream(infile)) {
+      fos.write(content);
+    }
+    assertEquals(content.length, infile.length());
+    File childDir = new File(TEST_DIR, childDirectoryName);
+    childDir.mkdirs();
+
+    Configuration conf = new Configuration();
+    ConfigUtil.addLink(conf, "/file", infile.toURI());
+    ConfigUtil.addLink(conf, "/dir", childDir.toURI());
+    String fileScheme = "file";
+    conf.set(String.format("fs.%s.impl", fileScheme),
+        ViewFileSystemOverloadScheme.class.getName());
+    conf.set(String
+        .format(FsConstants.FS_VIEWFS_OVERLOAD_SCHEME_TARGET_FS_IMPL_PATTERN,
+            fileScheme), LocalFileSystem.class.getName());
+    String fileUriStr = "file:///";
+    try (FileSystem vfs = FileSystem.get(new URI(fileUriStr), conf)) {
+      assertEquals(ViewFileSystemOverloadScheme.class, vfs.getClass());
+      FileStatus[] statuses = vfs.listStatus(new Path("/"));
+
+      FileSystem localFs = ((ViewFileSystemOverloadScheme) vfs)
+          .getRawFileSystem(new Path(fileUriStr), conf);
+      FileStatus fileStat = localFs.getFileStatus(new Path(infile.getPath()));
+      FileStatus dirStat = localFs.getFileStatus(new Path(childDir.getPath()));
+
+      for (FileStatus status : statuses) {
+        if (status.getPath().getName().equals(fileScheme)) {
+          assertEquals(fileStat.getPermission(), status.getPermission());
+        } else {
+          assertEquals(dirStat.getPermission(), status.getPermission());
+        }
+      }
+
+      localFs.setPermission(new Path(infile.getPath()),
+          FsPermission.valueOf("-rwxr--r--"));
+      localFs.setPermission(new Path(childDir.getPath()),
+          FsPermission.valueOf("-r--rwxr--"));
+
+      statuses = vfs.listStatus(new Path("/"));
+      for (FileStatus status : statuses) {
+        if (status.getPath().getName().equals(fileScheme)) {
+          assertEquals(FsPermission.valueOf("-rwxr--r--"),
+              status.getPermission());
+          assertFalse(status.isDirectory());
+        } else {
+          assertEquals(FsPermission.valueOf("-r--rwxr--"),
+              status.getPermission());
+          assertTrue(status.isDirectory());
+        }
+      }
+    }
+  }
+
+  @AfterClass
+  public static void cleanup() throws IOException {
+    FileUtil.fullyDelete(TEST_DIR);
+  }
+
+}
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java
index 29fcc22..7555745 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java
@@ -121,7 +121,7 @@ public class TestViewfsFileStatus {
     Configuration conf = new Configuration();
     ConfigUtil.addLink(conf, "/file", infile.toURI());
     ConfigUtil.addLink(conf, "/dir", childDir.toURI());
-
+    conf.setBoolean(Constants.CONFIG_VIEWFS_MOUNT_LINKS_AS_SYMLINKS, false);
     try (FileSystem vfs = FileSystem.get(FsConstants.VIEWFS_URI, conf)) {
       assertEquals(ViewFileSystem.class, vfs.getClass());
       FileStatus[] statuses = vfs.listStatus(new Path("/"));
@@ -148,9 +148,11 @@ public class TestViewfsFileStatus {
         if (status.getPath().getName().equals("file")) {
           assertEquals(FsPermission.valueOf("-rwxr--r--"),
               status.getPermission());
+          assertFalse(status.isDirectory());
         } else {
           assertEquals(FsPermission.valueOf("-r--rwxr--"),
               status.getPermission());
+          assertTrue(status.isDirectory());
         }
       }
     }
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFsOverloadScheme.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFsOverloadScheme.md
index 5fd8633..e65c545 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFsOverloadScheme.md
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/ViewFsOverloadScheme.md
@@ -28,7 +28,7 @@ View File System Overload Scheme
 
 ### Details
 
-The View File System Overload Scheme is an extension to the View File System. This will allow users to continue to use their existing fs.defaultFS configured scheme or any new scheme name instead of using scheme `viewfs`. Mount link configurations key, value formats are same as in [ViewFS Guide](./ViewFs.html). If a user wants to continue use the same fs.defaultFS and wants to have more mount points, then mount link configurations should have the current fs.defaultFS authority name as mo [...]
+The View File System Overload Scheme is an extension to the View File System. This will allow users to continue to use their existing fs.defaultFS configured scheme or any new scheme name instead of using scheme `viewfs`. Mount link configurations key, value formats are same as in [ViewFS Guide](./ViewFs.html). If a user wants to continue use the same fs.defaultFS and wants to have more mount points, then mount link configurations should have the current fs.defaultFS authority name as mo [...]
 
 Another important improvement with the ViewFileSystemOverloadScheme is, administrators need not copy the `mount-table.xml` configuration file to 1000s of client nodes. Instead they can keep the mount-table configuration file in a Hadoop compatible file system. So, keeping the configuration file in a central place makes administrators life easier as they can update mount-table in single place.
 
@@ -55,59 +55,59 @@ Here `<scheme>` should be same as the uri-scheme configured in fs.defautFS. For
 
 **Example 1:**
 
-If users want some of their existing cluster (`hdfs://mycluster`) data to mount with hdfs(`hdfs://mycluster`) and other object store clusters(`o3fs://bucket1.volume1.omhost/`, `s3a://bucket1/`), the following example configurations can show how to add mount links.
+If users want some of their existing cluster (`hdfs://cluster`) data to mount with hdfs(`hdfs://cluster`) and other object store clusters(`o3fs://bucket1.volume1.omhost/`, `s3a://bucket1/`), the following example configurations can show how to add mount links.
 
 
 ```xml
 <property>
-  <name>fs.viewfs.mounttable.Cluster./user</name>
-  <value>hdfs://mycluster/user</value>
+  <name>fs.viewfs.mounttable.cluster.link./user</name>
+  <value>hdfs://cluster/user</value>
 </property>
 
 <property>
-  <name>fs.viewfs.mounttable.Cluster./data</name>
+  <name>fs.viewfs.mounttable.cluster.link./data</name>
   <value>o3fs://bucket1.volume1/data</value>
 </property>
 
 <property>
-  <name>fs.viewfs.mounttable.Cluster./backup</name>
+  <name>fs.viewfs.mounttable.cluster.link./backup</name>
   <value>s3a://bucket1/backup/</value>
 </property>
 ```
 
 Let's consider the following operations to understand where these operations will be delegated based on mount links.
 
-  *Op1:* Create a file with the the path `hdfs://mycluster/user/fileA`, then physically this file will be created at `hdfs://mycluster/user/fileA`. This delegation happened based on the first configuration parameter in above configurations. Here `/user` mapped to `hdfs://mycluster/user/`.
+  *Op1:* Create a file with the the path `hdfs://cluster/user/fileA`, then physically this file will be created at `hdfs://cluster/user/fileA`. This delegation happened based on the first configuration parameter in above configurations. Here `/user` mapped to `hdfs://cluster/user/`.
 
-  *Op2:* Create a file the the path `hdfs://mycluster/data/datafile`, then this file will be created at `o3fs://bucket1.volume1.omhost/data/datafile`. This delegation happened based on second configurations parameter in above configurations. Here `/data` was mapped with `o3fs://bucket1.volume1.omhost/data/`.
+  *Op2:* Create a file the the path `hdfs://cluster/data/datafile`, then this file will be created at `o3fs://bucket1.volume1.omhost/data/datafile`. This delegation happened based on second configurations parameter in above configurations. Here `/data` was mapped with `o3fs://bucket1.volume1.omhost/data/`.
 
-  *Op3:* Create a file with the the path `hdfs://Cluster/backup/data.zip`, then  physically this file will be created at `s3a://bucket1/backup/data.zip`. This delegation happened based on the third configuration parameter in above configurations. Here `/backup` was mapped to `s3a://bucket1/backup/`.
+  *Op3:* Create a file with the the path `hdfs://cluster/backup/data.zip`, then  physically this file will be created at `s3a://bucket1/backup/data.zip`. This delegation happened based on the third configuration parameter in above configurations. Here `/backup` was mapped to `s3a://bucket1/backup/`.
 
 
 **Example 2:**
 
-If users want some of their existing cluster (`s3a://bucketA/`) data to mount with other hdfs cluster(`hdfs://Cluster`) and object store clusters(`o3fs://bucket1.volume1.omhost/`, `s3a://bucketA/`), the following example configurations can show how to add mount links.
+If users want some of their existing cluster (`s3a://bucketA/`) data to mount with other hdfs cluster(`hdfs://cluster`) and object store clusters(`o3fs://bucket1.volume1.omhost/`, `s3a://bucketA/`), the following example configurations can show how to add mount links.
 
 
 ```xml
 <property>
-  <name>fs.viewfs.mounttable.bucketA./user</name>
-  <value>hdfs://Cluster/user</value>
+  <name>fs.viewfs.mounttable.bucketA.link./user</name>
+  <value>hdfs://cluster/user</value>
 </property>
 
 <property>
-  <name>fs.viewfs.mounttable.bucketA./data</name>
+  <name>fs.viewfs.mounttable.bucketA.link./data</name>
   <value>o3fs://bucket1.volume1.omhost/data</value>
 </property>
 
 <property>
-  <name>fs.viewfs.mounttable.bucketA./salesDB</name>
+  <name>fs.viewfs.mounttable.bucketA.link./salesDB</name>
   <value>s3a://bucketA/salesDB/</value>
 </property>
 ```
 Let's consider the following operations to understand to where these operations will be delegated based on mount links.
 
- *Op1:* Create a file with the the path `s3a://bucketA/user/fileA`, then this file will be created physically at `hdfs://Cluster/user/fileA`. This delegation happened based on the first configuration parameter in above configurations. Here `/user` mapped to `hdfs://Cluster/user`.
+ *Op1:* Create a file with the the path `s3a://bucketA/user/fileA`, then this file will be created physically at `hdfs://cluster/user/fileA`. This delegation happened based on the first configuration parameter in above configurations. Here `/user` mapped to `hdfs://cluster/user`.
 
   *Op2:* Create a file the the path `s3a://bucketA/data/datafile`, then this file will be created at `o3fs://bucket1.volume1.omhost/data/datafile`. This delegation happened based on second configurations parameter in above configurations. Here `/data` was mapped with `o3fs://bucket1.volume1.omhost/data/`.
 
@@ -119,16 +119,18 @@ The following picture shows how the different schemes can be used in ViewFileSys
 
 <img src="./images/ViewFSOverloadScheme.png" width="1050" height="550"/>
 
+Note: In ViewFsOverloadScheme, by default the mount links will not be represented as symlinks. The permission bits and isDirectory value will be propagated from the target directory/file.
+
 ### Central Mount Table Configurations
 
-To enable central mount table configuration, we need to configure `fs.viewfs.mounttable.path` in `core-site.xml` with the value as the Hadoop compatible file system directory/file path, where the `mount-table-<versionNumber>.xml` file copied. Here versionNumber is an integer number and need to increase the version number and upload new file in same directory.
+To enable central mount table configuration, we need to configure `fs.viewfs.mounttable.path` in `core-site.xml` with the value as the Hadoop compatible file system directory/file path, where the `mount-table.<versionNumber>.xml` file copied. Here versionNumber is an integer number and need to increase the version number and upload new file in same directory.
 
-The ViewFileSystemOverloadScheme always loads the highest version number `mount-table-<versionNumber>.xml`. Please don't replace the file with same name. Always increment the version number to take new file picked by newly initializing clients. Why we don't recommend to replace the files is that, some client might have already opened the connections to old mount-table files already and in middle of loading configuration files, and replacing files can make them fail.
+The ViewFileSystemOverloadScheme always loads the highest version number `mount-table.<versionNumber>.xml`. Please don't replace the file with same name. Always increment the version number to take new file picked by newly initializing clients. Why we don't recommend to replace the files is that, some client might have already opened the connections to old mount-table files already and in middle of loading configuration files, and replacing files can make them fail.
 
 ```xml
 <property>
   <name>fs.viewfs.mounttable.path</name>
-  <value>hdfs://Cluster/config/mount-table-dir</value>
+  <value>hdfs://cluster/config/mount-table-dir</value>
 </property>
 ```
  If you are sure, you will never do updates to mount-table file, you can also configure file path directly like below. If you configure file path, it will not check any highest version number loading. Whatever file configured it will be loaded. However file name format should be same.
@@ -136,12 +138,12 @@ The ViewFileSystemOverloadScheme always loads the highest version number `mount-
 ```xml
 <property>
   <name>fs.viewfs.mounttable.path</name>
-  <value>hdfs://Cluster/config/mount-table-dir/mount-table-<versionNumber>.xml</value>
+  <value>hdfs://cluster/config/mount-table-dir/mount-table.<versionNumber>.xml</value>
 </property>
 ```
 Note: we recommend not to configure mount-links in `core-site.xml` if you configure above valid path. Otherwise both mount links will be mixed and can lead to a confused behavior.
 
-If you copy the `mount-table-<versionNumber>.xml`, you may consider having big replication factor depending on your cluster size. So, that file will be available locally to majority of clients as applications(MR/YARN/HBASE..etc) use locality on HDFS when reading `mount-table-<versionNumber>.xml`.
+If you copy the `mount-table.<versionNumber>.xml`, you may consider having big replication factor depending on your cluster size. So, that file will be available locally to majority of clients as applications(MR/YARN/HBASE..etc) use locality on HDFS when reading `mount-table.<versionNumber>.xml`.
 
 DFSAdmin commands with View File System Overload Scheme
 -------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java
index 03c29c9..e7e74d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeHdfsFileSystemContract.java
@@ -116,6 +116,11 @@ public class TestViewFileSystemOverloadSchemeHdfsFileSystemContract
     assumeTrue(rootDirTestEnabled());
     Path dir = path("/");
     Path child = path("/FileSystemContractBaseTest");
+    try (FileSystem dfs = ((ViewFileSystemOverloadScheme) fs).getRawFileSystem(
+        new Path(conf.get(CommonConfigurationKeys.FS_DEFAULT_NAME_KEY), "/"),
+        conf)) {
+      dfs.mkdirs(child);
+    }
     assertListStatusFinds(dir, child);
   }
 
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java
index 3860fa4..417a80c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemOverloadSchemeWithHdfsScheme.java
@@ -191,8 +191,17 @@ public class TestViewFileSystemOverloadSchemeWithHdfsScheme {
         new String[] {hdfsTargetPath.toUri().toString(),
             localTargetDir.toURI().toString() },
         conf);
+    try (DistributedFileSystem dfs = new DistributedFileSystem()) {
+      dfs.initialize(defaultFSURI, conf);
+      dfs.mkdirs(hdfsTargetPath);
+    }
 
+    try (RawLocalFileSystem lfs = new RawLocalFileSystem()) {
+      lfs.initialize(localTargetDir.toURI(), conf);
+      lfs.mkdirs(new Path(localTargetDir.toURI()));
+    }
     try (FileSystem fs = FileSystem.get(conf)) {
+      fs.mkdirs(hdfsTargetPath);
       FileStatus[] ls = fs.listStatus(new Path("/"));
       Assert.assertEquals(2, ls.length);
       String lsPath1 =


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