Просмотр исходного кода

HDFS-11058. Implement 'hadoop fs -df' command for ViewFileSystem. Contributed by Manoj Govindassamy.

Andrew Wang 8 лет назад
Родитель
Сommit
dd98a8005a

+ 95 - 29
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/shell/FsUsage.java

@@ -20,19 +20,24 @@ package org.apache.hadoop.fs.shell;
 
 import java.io.IOException;
 import java.io.PrintStream;
+import java.net.URI;
 import java.util.ArrayList;
 import java.util.LinkedList;
 import java.util.List;
+import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.ContentSummary;
 import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ViewFileSystem;
+import org.apache.hadoop.fs.viewfs.ViewFileSystemUtil;
 import org.apache.hadoop.util.StringUtils;
 
-/** Base class for commands related to viewing filesystem usage, such as
- * du and df
+/**
+ * Base class for commands related to viewing filesystem usage,
+ * such as du and df.
  */
 @InterfaceAudience.Private
 @InterfaceStability.Evolving
@@ -44,15 +49,27 @@ class FsUsage extends FsCommand {
     factory.addClass(Dus.class, "-dus");
   }
 
-  protected boolean humanReadable = false;
-  protected TableBuilder usagesTable;
-  
+  private boolean humanReadable = false;
+  private TableBuilder usagesTable;
+
   protected String formatSize(long size) {
     return humanReadable
         ? StringUtils.TraditionalBinaryPrefix.long2String(size, "", 1)
         : String.valueOf(size);
   }
 
+  public TableBuilder getUsagesTable() {
+    return usagesTable;
+  }
+
+  public void setUsagesTable(TableBuilder usagesTable) {
+    this.usagesTable = usagesTable;
+  }
+
+  public void setHumanReadable(boolean humanReadable) {
+    this.humanReadable = humanReadable;
+  }
+
   /** Show the size of a partition in the filesystem */
   public static class Df extends FsUsage {
     public static final String NAME = "df";
@@ -70,38 +87,74 @@ class FsUsage extends FsCommand {
     throws IOException {
       CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, "h");
       cf.parse(args);
-      humanReadable = cf.getOpt("h");
+      setHumanReadable(cf.getOpt("h"));
       if (args.isEmpty()) args.add(Path.SEPARATOR);
     }
 
     @Override
     protected void processArguments(LinkedList<PathData> args)
     throws IOException {
-      usagesTable = new TableBuilder(
-          "Filesystem", "Size", "Used", "Available", "Use%");
-      usagesTable.setRightAlign(1, 2, 3, 4);
-      
+      setUsagesTable(new TableBuilder(
+          "Filesystem", "Size", "Used", "Available", "Use%", "Mounted on"));
+      getUsagesTable().setRightAlign(1, 2, 3, 4);
+
       super.processArguments(args);
-      if (!usagesTable.isEmpty()) {
-        usagesTable.printToStream(out);
+      if (!getUsagesTable().isEmpty()) {
+        getUsagesTable().printToStream(out);
       }
     }
 
-    @Override
-    protected void processPath(PathData item) throws IOException {
-      FsStatus fsStats = item.fs.getStatus(item.path);
-      long size = fsStats.getCapacity();
-      long used = fsStats.getUsed();
-      long free = fsStats.getRemaining();
-
-      usagesTable.addRow(
-          item.fs.getUri(),
+    /**
+     * Add a new row to the usages table for the given FileSystem URI.
+     *
+     * @param uri - FileSystem URI
+     * @param fsStatus - FileSystem status
+     * @param mountedOnPath - FileSystem mounted on path
+     */
+    private void addToUsagesTable(URI uri, FsStatus fsStatus,
+        String mountedOnPath) {
+      long size = fsStatus.getCapacity();
+      long used = fsStatus.getUsed();
+      long free = fsStatus.getRemaining();
+      getUsagesTable().addRow(
+          uri,
           formatSize(size),
           formatSize(used),
           formatSize(free),
-          StringUtils.formatPercent((double)used/(double)size, 0)
+          StringUtils.formatPercent((double) used / (double) size, 0),
+          mountedOnPath
       );
     }
+
+    @Override
+    protected void processPath(PathData item) throws IOException {
+      if (ViewFileSystemUtil.isViewFileSystem(item.fs)) {
+        ViewFileSystem viewFileSystem = (ViewFileSystem) item.fs;
+        Map<ViewFileSystem.MountPoint, FsStatus>  fsStatusMap =
+            ViewFileSystemUtil.getStatus(viewFileSystem, item.path);
+
+        for (Map.Entry<ViewFileSystem.MountPoint, FsStatus> entry :
+            fsStatusMap.entrySet()) {
+          ViewFileSystem.MountPoint viewFsMountPoint = entry.getKey();
+          FsStatus fsStatus = entry.getValue();
+
+          // Add the viewfs mount point status to report
+          URI[] mountPointFileSystemURIs =
+              viewFsMountPoint.getTargetFileSystemURIs();
+          // Since LinkMerge is not supported yet, we
+          // should ideally see mountPointFileSystemURIs
+          // array with only one element.
+          addToUsagesTable(mountPointFileSystemURIs[0],
+              fsStatus, viewFsMountPoint.getMountedOnPath().toString());
+        }
+      } else {
+        // Hide the columns specific to ViewFileSystem
+        getUsagesTable().setColumnHide(5, true);
+        FsStatus fsStatus = item.fs.getStatus(item.path);
+        addToUsagesTable(item.fs.getUri(), fsStatus, "/");
+      }
+    }
+
   }
 
   /** show disk usage */
@@ -128,7 +181,7 @@ class FsUsage extends FsCommand {
     protected void processOptions(LinkedList<String> args) throws IOException {
       CommandFormat cf = new CommandFormat(0, Integer.MAX_VALUE, "h", "s", "x");
       cf.parse(args);
-      humanReadable = cf.getOpt("h");
+      setHumanReadable(cf.getOpt("h"));
       summary = cf.getOpt("s");
       excludeSnapshots = cf.getOpt("x");
       if (args.isEmpty()) args.add(Path.CUR_DIR);
@@ -137,10 +190,10 @@ class FsUsage extends FsCommand {
     @Override
     protected void processArguments(LinkedList<PathData> args)
         throws IOException {
-      usagesTable = new TableBuilder(3);
+      setUsagesTable(new TableBuilder(3));
       super.processArguments(args);
-      if (!usagesTable.isEmpty()) {
-        usagesTable.printToStream(out);
+      if (!getUsagesTable().isEmpty()) {
+        getUsagesTable().printToStream(out);
       }
     }
 
@@ -163,7 +216,8 @@ class FsUsage extends FsCommand {
         length -= contentSummary.getSnapshotLength();
         spaceConsumed -= contentSummary.getSnapshotSpaceConsumed();
       }
-      usagesTable.addRow(formatSize(length), formatSize(spaceConsumed), item);
+      getUsagesTable().addRow(formatSize(length),
+          formatSize(spaceConsumed), item);
     }
   }
   /** show disk usage summary */
@@ -191,6 +245,7 @@ class FsUsage extends FsCommand {
     protected List<String[]> rows;
     protected int[] widths;
     protected boolean[] rightAlign;
+    private boolean[] hide;
     
     /**
      * Create a table w/o headers
@@ -200,6 +255,7 @@ class FsUsage extends FsCommand {
       rows = new ArrayList<String[]>();
       widths = new int[columns];
       rightAlign = new boolean[columns];
+      hide = new boolean[columns];
     }
 
     /**
@@ -219,7 +275,14 @@ class FsUsage extends FsCommand {
     public void setRightAlign(int ... indexes) {
       for (int i : indexes) rightAlign[i] = true;
     }
-    
+
+    /**
+     * Hide the given column index
+     */
+    public void setColumnHide(int columnIndex, boolean hideCol) {
+      hide[columnIndex] = hideCol;
+    }
+
     /**
      * Add a row of objects to the table
      * @param objects the values
@@ -234,7 +297,7 @@ class FsUsage extends FsCommand {
     }
 
     /**
-     * Render the table to a stream 
+     * Render the table to a stream.
      * @param out PrintStream for output
      */
     public void printToStream(PrintStream out) {
@@ -242,6 +305,9 @@ class FsUsage extends FsCommand {
 
       StringBuilder fmt = new StringBuilder();      
       for (int i=0; i < widths.length; i++) {
+        if (hide[i]) {
+          continue;
+        }
         if (fmt.length() != 0) fmt.append("  ");
         if (rightAlign[i]) {
           fmt.append("%"+widths[i]+"s");

+ 36 - 20
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystem.java

@@ -33,7 +33,6 @@ import java.util.Map;
 import java.util.Set;
 import java.util.Map.Entry;
 
-import com.google.common.annotations.VisibleForTesting;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.conf.Configuration;
@@ -49,6 +48,7 @@ import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.FsConstants;
 import org.apache.hadoop.fs.FsServerDefaults;
+import org.apache.hadoop.fs.FsStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
@@ -90,34 +90,35 @@ public class ViewFileSystem extends FileSystem {
     return readOnlyMountTable(operation, p.toString());
   }
 
-  static public class MountPoint {
+  /**
+   * MountPoint representation built from the configuration.
+   */
+  public static class MountPoint {
+
     /**
-     *  The source of the mount.
+     * The mounted on path location.
      */
-    private Path src;
+    private final Path mountedOnPath;
 
     /**
-     * One or more targets of the mount.
-     * Multiple targets imply MergeMount.
+     * Array of target FileSystem URIs.
      */
-    private URI[] targets;
+    private final URI[] targetFileSystemURIs;
 
-    MountPoint(Path srcPath, URI[] targetURIs) {
-      src = srcPath;
-      targets = targetURIs;
+    MountPoint(Path srcPath, URI[] targetFs) {
+      mountedOnPath = srcPath;
+      targetFileSystemURIs = targetFs;
     }
 
-    @VisibleForTesting
-    Path getSrc() {
-      return src;
+    public Path getMountedOnPath() {
+      return mountedOnPath;
     }
 
-    @VisibleForTesting
-    URI[] getTargets() {
-      return targets;
+    public URI[] getTargetFileSystemURIs() {
+      return targetFileSystemURIs;
     }
   }
-  
+
   final long creationTime; // of the the mount table
   final UserGroupInformation ugi; // the user/group of user who created mtable
   URI myUri;
@@ -134,7 +135,7 @@ public class ViewFileSystem extends FileSystem {
    * @param p path
    * @return path-part of the Path p
    */
-  private String getUriPath(final Path p) {
+  String getUriPath(final Path p) {
     checkPath(p);
     return makeAbsolute(p).toUri().getPath();
   }
@@ -732,8 +733,8 @@ public class ViewFileSystem extends FileSystem {
     
     MountPoint[] result = new MountPoint[mountPoints.size()];
     for ( int i = 0; i < mountPoints.size(); ++i ) {
-      result[i] = new MountPoint(new Path(mountPoints.get(i).src), 
-                              mountPoints.get(i).target.targetDirLinkList);
+      result[i] = new MountPoint(new Path(mountPoints.get(i).src),
+          mountPoints.get(i).target.targetDirLinkList);
     }
     return result;
   }
@@ -833,6 +834,21 @@ public class ViewFileSystem extends FileSystem {
     return trashRoots;
   }
 
+  @Override
+  public FsStatus getStatus() throws IOException {
+    return getStatus(null);
+  }
+
+  @Override
+  public FsStatus getStatus(Path p) throws IOException {
+    if (p == null) {
+      p = InodeTree.SlashPath;
+    }
+    InodeTree.ResolveResult<FileSystem> res = fsState.resolve(
+        getUriPath(p), true);
+    return res.targetFileSystem.getStatus(p);
+  }
+
   /**
    * An instance of this class represents an internal dir of the viewFs
    * that is internal dir of the mount table.

+ 164 - 0
hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/viewfs/ViewFileSystemUtil.java

@@ -0,0 +1,164 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.fs.viewfs;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FsConstants;
+import org.apache.hadoop.fs.FsStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.UnsupportedFileSystemException;
+import org.apache.hadoop.fs.viewfs.ViewFileSystem.MountPoint;
+
+/**
+ * Utility APIs for ViewFileSystem.
+ */
+@InterfaceAudience.Public
+@InterfaceStability.Evolving
+public final class ViewFileSystemUtil {
+
+  private ViewFileSystemUtil() {
+    // Private Constructor
+  }
+
+  /**
+   * Check if the FileSystem is a ViewFileSystem.
+   *
+   * @param fileSystem
+   * @return true if the fileSystem is ViewFileSystem
+   */
+  public static boolean isViewFileSystem(final FileSystem fileSystem) {
+    return fileSystem.getScheme().equals(FsConstants.VIEWFS_SCHEME);
+  }
+
+  /**
+   * Get FsStatus for all ViewFsMountPoints matching path for the given
+   * ViewFileSystem.
+   *
+   * Say ViewFileSystem has following mount points configured
+   *  (1) hdfs://NN0_host:port/sales mounted on /dept/sales
+   *  (2) hdfs://NN1_host:port/marketing mounted on /dept/marketing
+   *  (3) hdfs://NN2_host:port/eng_usa mounted on /dept/eng/usa
+   *  (4) hdfs://NN3_host:port/eng_asia mounted on /dept/eng/asia
+   *
+   * For the above config, here is a sample list of paths and their matching
+   * mount points while getting FsStatus
+   *
+   *  Path                  Description                      Matching MountPoint
+   *
+   *  "/"                   Root ViewFileSystem lists all    (1), (2), (3), (4)
+   *                         mount points.
+   *
+   *  "/dept"               Not a mount point, but a valid   (1), (2), (3), (4)
+   *                         internal dir in the mount tree
+   *                         and resolved down to "/" path.
+   *
+   *  "/dept/sales"         Matches a mount point            (1)
+   *
+   *  "/dept/sales/india"   Path is over a valid mount point (1)
+   *                         and resolved down to
+   *                         "/dept/sales"
+   *
+   *  "/dept/eng"           Not a mount point, but a valid   (1), (2), (3), (4)
+   *                         internal dir in the mount tree
+   *                         and resolved down to "/" path.
+   *
+   *  "/erp"                Doesn't match or leads to or
+   *                         over any valid mount points     None
+   *
+   *
+   * @param fileSystem - ViewFileSystem on which mount point exists
+   * @param path - URI for which FsStatus is requested
+   * @return Map of ViewFsMountPoint and FsStatus
+   */
+  public static Map<MountPoint, FsStatus> getStatus(
+      FileSystem fileSystem, Path path) throws IOException {
+    if (!isViewFileSystem(fileSystem)) {
+      throw new UnsupportedFileSystemException("FileSystem '"
+          + fileSystem.getUri() + "'is not a ViewFileSystem.");
+    }
+    ViewFileSystem viewFileSystem = (ViewFileSystem) fileSystem;
+    String viewFsUriPath = viewFileSystem.getUriPath(path);
+    boolean isPathOverMountPoint = false;
+    boolean isPathLeadingToMountPoint = false;
+    boolean isPathIncludesAllMountPoint = false;
+    Map<MountPoint, FsStatus> mountPointMap = new HashMap<>();
+    for (MountPoint mountPoint : viewFileSystem.getMountPoints()) {
+      String[] mountPointPathComponents = InodeTree.breakIntoPathComponents(
+          mountPoint.getMountedOnPath().toString());
+      String[] incomingPathComponents =
+          InodeTree.breakIntoPathComponents(viewFsUriPath);
+
+      int pathCompIndex;
+      for (pathCompIndex = 0; pathCompIndex < mountPointPathComponents.length &&
+          pathCompIndex < incomingPathComponents.length; pathCompIndex++) {
+        if (!mountPointPathComponents[pathCompIndex].equals(
+            incomingPathComponents[pathCompIndex])) {
+          break;
+        }
+      }
+
+      if (pathCompIndex >= mountPointPathComponents.length) {
+        // Patch matches or over a valid mount point
+        isPathOverMountPoint = true;
+        mountPointMap.clear();
+        updateMountPointFsStatus(viewFileSystem, mountPointMap, mountPoint,
+            new Path(viewFsUriPath));
+        break;
+      } else {
+        if (pathCompIndex > 1) {
+          // Path is in the mount tree
+          isPathLeadingToMountPoint = true;
+        } else if (incomingPathComponents.length <= 1) {
+          // Special case of "/" path
+          isPathIncludesAllMountPoint = true;
+        }
+        updateMountPointFsStatus(viewFileSystem, mountPointMap, mountPoint,
+            mountPoint.getMountedOnPath());
+      }
+    }
+
+    if (!isPathOverMountPoint && !isPathLeadingToMountPoint &&
+        !isPathIncludesAllMountPoint) {
+      throw new NotInMountpointException(path, "getStatus");
+    }
+    return mountPointMap;
+  }
+
+  /**
+   * Update FsStatus for the given the mount point.
+   *
+   * @param viewFileSystem
+   * @param mountPointMap
+   * @param mountPoint
+   * @param path
+   */
+  private static void updateMountPointFsStatus(
+      final ViewFileSystem viewFileSystem,
+      final Map<MountPoint, FsStatus> mountPointMap,
+      final MountPoint mountPoint, final Path path) throws IOException {
+    FsStatus fsStatus = viewFileSystem.getStatus(path);
+    mountPointMap.put(mountPoint, fsStatus);
+  }
+
+}

+ 5 - 4
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/TestViewfsFileStatus.java

@@ -86,19 +86,20 @@ public class TestViewfsFileStatus {
   // .getFileChecksum with res.remainingPath and not with f
   @Test
   public void testGetFileChecksum() throws IOException {
+    final Path path = new Path("/tmp/someFile");
     FileSystem mockFS = Mockito.mock(FileSystem.class);
     InodeTree.ResolveResult<FileSystem> res =
       new InodeTree.ResolveResult<FileSystem>(null, mockFS , null,
         new Path("someFile"));
     @SuppressWarnings("unchecked")
     InodeTree<FileSystem> fsState = Mockito.mock(InodeTree.class);
-    Mockito.when(fsState.resolve("/tmp/someFile", true)).thenReturn(res);
+    Mockito.when(fsState.resolve(path.toString(), true)).thenReturn(res);
     ViewFileSystem vfs = Mockito.mock(ViewFileSystem.class);
     vfs.fsState = fsState;
 
-    Mockito.when(vfs.getFileChecksum(new Path("/tmp/someFile")))
-      .thenCallRealMethod();
-    vfs.getFileChecksum(new Path("/tmp/someFile"));
+    Mockito.when(vfs.getFileChecksum(path)).thenCallRealMethod();
+    Mockito.when(vfs.getUriPath(path)).thenCallRealMethod();
+    vfs.getFileChecksum(path);
 
     Mockito.verify(mockFS).getFileChecksum(new Path("someFile"));
   }

+ 71 - 3
hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/fs/viewfs/ViewFileSystemBaseTest.java

@@ -24,8 +24,9 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.List;
 import java.util.ArrayList;
+import java.util.Map;
+import java.util.Map.Entry;
 
-import com.google.common.base.Joiner;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.BlockLocation;
 import org.apache.hadoop.fs.BlockStoragePolicySpi;
@@ -33,6 +34,7 @@ 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.FsStatus;
 import org.apache.hadoop.fs.LocatedFileStatus;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RemoteIterator;
@@ -153,8 +155,8 @@ abstract public class ViewFileSystemBaseTest {
     ViewFileSystem viewfs = (ViewFileSystem) fsView;
     MountPoint[] mountPoints = viewfs.getMountPoints();
     for (MountPoint mountPoint : mountPoints) {
-      LOG.info("MountPoint: " + mountPoint.getSrc() + " => "
-          + Joiner.on(",").join(mountPoint.getTargets()));
+      LOG.info("MountPoint: " + mountPoint.getMountedOnPath() + " => "
+          + mountPoint.getTargetFileSystemURIs()[0]);
     }
     Assert.assertEquals(getExpectedMountPoints(), mountPoints.length); 
   }
@@ -1020,4 +1022,70 @@ abstract public class ViewFileSystemBaseTest {
     // target mounted FileSystem.
     Assert.assertTrue("", fsView.getTrashRoots(true).size() > 0);
   }
+
+  @Test(expected = NotInMountpointException.class)
+  public void testViewFileSystemUtil() throws Exception {
+    Configuration newConf = new Configuration(conf);
+
+    FileSystem fileSystem = FileSystem.get(FsConstants.LOCAL_FS_URI,
+        newConf);
+    Assert.assertFalse("Unexpected FileSystem: " + fileSystem,
+        ViewFileSystemUtil.isViewFileSystem(fileSystem));
+
+    fileSystem = FileSystem.get(FsConstants.VIEWFS_URI,
+        newConf);
+    Assert.assertTrue("Unexpected FileSystem: " + fileSystem,
+        ViewFileSystemUtil.isViewFileSystem(fileSystem));
+
+    // Case 1: Verify FsStatus of root path returns all MountPoints status.
+    Map<MountPoint, FsStatus> mountPointFsStatusMap =
+        ViewFileSystemUtil.getStatus(fileSystem, InodeTree.SlashPath);
+    Assert.assertEquals(getExpectedMountPoints(), mountPointFsStatusMap.size());
+
+    // Case 2: Verify FsStatus of an internal dir returns all
+    // MountPoints status.
+    mountPointFsStatusMap =
+        ViewFileSystemUtil.getStatus(fileSystem, new Path("/internalDir"));
+    Assert.assertEquals(getExpectedMountPoints(), mountPointFsStatusMap.size());
+
+    // Case 3: Verify FsStatus of a matching MountPoint returns exactly
+    // the corresponding MountPoint status.
+    mountPointFsStatusMap =
+        ViewFileSystemUtil.getStatus(fileSystem, new Path("/user"));
+    Assert.assertEquals(1, mountPointFsStatusMap.size());
+    for (Entry<MountPoint, FsStatus> entry : mountPointFsStatusMap.entrySet()) {
+      Assert.assertEquals(entry.getKey().getMountedOnPath().toString(),
+          "/user");
+    }
+
+    // Case 4: Verify FsStatus of a path over a MountPoint returns the
+    // corresponding MountPoint status.
+    mountPointFsStatusMap =
+        ViewFileSystemUtil.getStatus(fileSystem, new Path("/user/cloud"));
+    Assert.assertEquals(1, mountPointFsStatusMap.size());
+    for (Entry<MountPoint, FsStatus> entry : mountPointFsStatusMap.entrySet()) {
+      Assert.assertEquals(entry.getKey().getMountedOnPath().toString(),
+          "/user");
+    }
+
+    // Case 5: Verify FsStatus of any level of an internal dir
+    // returns all MountPoints status.
+    mountPointFsStatusMap =
+        ViewFileSystemUtil.getStatus(fileSystem,
+            new Path("/internalDir/internalDir2"));
+    Assert.assertEquals(getExpectedMountPoints(), mountPointFsStatusMap.size());
+
+    // Case 6: Verify FsStatus of a MountPoint URI returns
+    // the MountPoint status.
+    mountPointFsStatusMap =
+        ViewFileSystemUtil.getStatus(fileSystem, new Path("viewfs:/user/"));
+    Assert.assertEquals(1, mountPointFsStatusMap.size());
+    for (Entry<MountPoint, FsStatus> entry : mountPointFsStatusMap.entrySet()) {
+      Assert.assertEquals(entry.getKey().getMountedOnPath().toString(),
+          "/user");
+    }
+
+    // Case 7: Verify FsStatus of a non MountPoint path throws exception
+    ViewFileSystemUtil.getStatus(fileSystem, new Path("/non-existing"));
+  }
 }

+ 24 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/fs/viewfs/TestViewFileSystemHdfs.java

@@ -194,4 +194,28 @@ public class TestViewFileSystemHdfs extends ViewFileSystemBaseTest {
     Assert.assertTrue("ViewFileSystem trash roots should include EZ zone trash",
         (fsView.getTrashRoots(true).size() == 2));
   }
+
+  @Test
+  public void testDf() throws Exception {
+    Configuration newConf = new Configuration(conf);
+
+    // Verify if DF on non viewfs produces output as before, that is
+    // without "Mounted On" header.
+    DFSTestUtil.FsShellRun("-df", 0, "Use%" + System.lineSeparator(), newConf);
+
+    // Setting the default Fs to viewfs
+    newConf.set("fs.default.name", "viewfs:///");
+
+    // Verify if DF on viewfs produces a new header "Mounted on"
+    DFSTestUtil.FsShellRun("-df /user", 0, "Mounted on", newConf);
+
+    DFSTestUtil.FsShellRun("-df viewfs:///user", 0, "/user", newConf);
+    DFSTestUtil.FsShellRun("-df /user3", 1, "/user3", newConf);
+    DFSTestUtil.FsShellRun("-df /user2/abc", 1, "No such file or directory",
+        newConf);
+    DFSTestUtil.FsShellRun("-df /user2/", 0, "/user2", newConf);
+    DFSTestUtil.FsShellRun("-df /internalDir", 0, "/internalDir", newConf);
+    DFSTestUtil.FsShellRun("-df /", 0, null, newConf);
+    DFSTestUtil.FsShellRun("-df", 0, null, newConf);
+  }
 }