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 in...@apache.org on 2019/02/20 22:21:38 UTC

[hadoop] 12/41: HDFS-14085. RBF: LS command for root shows wrong owner and permission information. Contributed by Ayush Saxena.

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

inigoiri pushed a commit to branch HDFS-13891
in repository https://gitbox.apache.org/repos/asf/hadoop.git

commit 16b8f759a1eafba5654767b3344e5b1a4740d851
Author: Surendra Singh Lilhore <su...@apache.org>
AuthorDate: Tue Dec 4 12:23:56 2018 +0530

    HDFS-14085. RBF: LS command for root shows wrong owner and permission information. Contributed by Ayush Saxena.
---
 .../server/federation/router/FederationUtil.java   |  23 +-
 .../federation/router/RouterClientProtocol.java    |  29 +-
 .../federation/router/TestRouterMountTable.java    | 307 ++++++++++++++++-----
 3 files changed, 278 insertions(+), 81 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
index f8c7a9b..f0d9168 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/FederationUtil.java
@@ -27,6 +27,7 @@ import java.net.URLConnection;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.server.federation.resolver.ActiveNamenodeResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.store.StateStoreService;
@@ -205,4 +206,24 @@ public final class FederationUtil {
     return path.charAt(parent.length()) == Path.SEPARATOR_CHAR
         || parent.equals(Path.SEPARATOR);
   }
-}
+
+  /**
+   * Add the the number of children for an existing HdfsFileStatus object.
+   * @param dirStatus HdfsfileStatus object.
+   * @param children number of children to be added.
+   * @return HdfsFileStatus with the number of children specified.
+   */
+  public static HdfsFileStatus updateMountPointStatus(HdfsFileStatus dirStatus,
+      int children) {
+    return new HdfsFileStatus.Builder().atime(dirStatus.getAccessTime())
+        .blocksize(dirStatus.getBlockSize()).children(children)
+        .ecPolicy(dirStatus.getErasureCodingPolicy())
+        .feInfo(dirStatus.getFileEncryptionInfo()).fileId(dirStatus.getFileId())
+        .group(dirStatus.getGroup()).isdir(dirStatus.isDir())
+        .length(dirStatus.getLen()).mtime(dirStatus.getModificationTime())
+        .owner(dirStatus.getOwner()).path(dirStatus.getLocalNameInBytes())
+        .perm(dirStatus.getPermission()).replication(dirStatus.getReplication())
+        .storagePolicy(dirStatus.getStoragePolicy())
+        .symlink(dirStatus.getSymlinkInBytes()).build();
+  }
+}
\ No newline at end of file
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
index 81717ca..2089c57 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.hdfs.server.federation.router;
 
+import static org.apache.hadoop.hdfs.server.federation.router.FederationUtil.updateMountPointStatus;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.crypto.CryptoProtocolVersion;
 import org.apache.hadoop.fs.BatchedRemoteIterator;
@@ -669,7 +670,6 @@ public class RouterClientProtocol implements ClientProtocol {
         if (dates != null && dates.containsKey(child)) {
           date = dates.get(child);
         }
-        // TODO add number of children
         HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
 
         // This may overwrite existing listing entries with the mount point
@@ -1663,12 +1663,13 @@ public class RouterClientProtocol implements ClientProtocol {
     // Get the file info from everybody
     Map<RemoteLocation, HdfsFileStatus> results =
         rpcClient.invokeConcurrent(locations, method, HdfsFileStatus.class);
-
+    int children=0;
     // We return the first file
     HdfsFileStatus dirStatus = null;
     for (RemoteLocation loc : locations) {
       HdfsFileStatus fileStatus = results.get(loc);
       if (fileStatus != null) {
+        children += fileStatus.getChildrenNum();
         if (!fileStatus.isDirectory()) {
           return fileStatus;
         } else if (dirStatus == null) {
@@ -1676,7 +1677,10 @@ public class RouterClientProtocol implements ClientProtocol {
         }
       }
     }
-    return dirStatus;
+    if (dirStatus != null) {
+      return updateMountPointStatus(dirStatus, children);
+    }
+    return null;
   }
 
   /**
@@ -1732,12 +1736,23 @@ public class RouterClientProtocol implements ClientProtocol {
     String group = this.superGroup;
     if (subclusterResolver instanceof MountTableResolver) {
       try {
+        String mName = name.startsWith("/") ? name : "/" + name;
         MountTableResolver mountTable = (MountTableResolver) subclusterResolver;
-        MountTable entry = mountTable.getMountPoint(name);
+        MountTable entry = mountTable.getMountPoint(mName);
         if (entry != null) {
-          permission = entry.getMode();
-          owner = entry.getOwnerName();
-          group = entry.getGroupName();
+          HdfsFileStatus fInfo = getFileInfoAll(entry.getDestinations(),
+              new RemoteMethod("getFileInfo", new Class<?>[] {String.class},
+                  new RemoteParam()));
+          if (fInfo != null) {
+            permission = fInfo.getPermission();
+            owner = fInfo.getOwner();
+            group = fInfo.getGroup();
+            childrenNum = fInfo.getChildrenNum();
+          } else {
+            permission = entry.getMode();
+            owner = entry.getOwnerName();
+            group = entry.getGroupName();
+          }
         }
       } catch (IOException e) {
         LOG.error("Cannot get mount point: {}", e.getMessage());
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java
index d2b78d3..9538d71 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java
@@ -23,6 +23,7 @@ import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
@@ -60,18 +61,21 @@ import org.junit.Test;
 public class TestRouterMountTable {
 
   private static StateStoreDFSCluster cluster;
-  private static NamenodeContext nnContext;
+  private static NamenodeContext nnContext0;
+  private static NamenodeContext nnContext1;
   private static RouterContext routerContext;
   private static MountTableResolver mountTable;
   private static ClientProtocol routerProtocol;
   private static long startTime;
+  private static FileSystem nnFs0;
+  private static FileSystem nnFs1;
 
   @BeforeClass
   public static void globalSetUp() throws Exception {
     startTime = Time.now();
 
     // Build and start a federated cluster
-    cluster = new StateStoreDFSCluster(false, 1);
+    cluster = new StateStoreDFSCluster(false, 2);
     Configuration conf = new RouterConfigBuilder()
         .stateStore()
         .admin()
@@ -83,7 +87,10 @@ public class TestRouterMountTable {
     cluster.waitClusterUp();
 
     // Get the end points
-    nnContext = cluster.getRandomNamenode();
+    nnContext0 = cluster.getNamenode("ns0", null);
+    nnContext1 = cluster.getNamenode("ns1", null);
+    nnFs0 = nnContext0.getFileSystem();
+    nnFs1 = nnContext1.getFileSystem();
     routerContext = cluster.getRandomRouter();
     Router router = routerContext.getRouter();
     routerProtocol = routerContext.getClient().getNamenode();
@@ -129,12 +136,11 @@ public class TestRouterMountTable {
     assertTrue(addMountTable(regularEntry));
 
     // Create a folder which should show in all locations
-    final FileSystem nnFs = nnContext.getFileSystem();
     final FileSystem routerFs = routerContext.getFileSystem();
     assertTrue(routerFs.mkdirs(new Path("/regular/newdir")));
 
     FileStatus dirStatusNn =
-        nnFs.getFileStatus(new Path("/testdir/newdir"));
+        nnFs0.getFileStatus(new Path("/testdir/newdir"));
     assertTrue(dirStatusNn.isDirectory());
     FileStatus dirStatusRegular =
         routerFs.getFileStatus(new Path("/regular/newdir"));
@@ -179,93 +185,248 @@ public class TestRouterMountTable {
    */
   @Test
   public void testListFilesTime() throws Exception {
-    // Add mount table entry
-    MountTable addEntry = MountTable.newInstance(
-        "/testdir", Collections.singletonMap("ns0", "/testdir"));
-    assertTrue(addMountTable(addEntry));
-    addEntry = MountTable.newInstance(
-        "/testdir2", Collections.singletonMap("ns0", "/testdir2"));
-    assertTrue(addMountTable(addEntry));
-    addEntry = MountTable.newInstance(
-        "/testdir/subdir", Collections.singletonMap("ns0", "/testdir/subdir"));
-    assertTrue(addMountTable(addEntry));
-    addEntry = MountTable.newInstance(
-        "/testdir3/subdir1", Collections.singletonMap("ns0", "/testdir3"));
-    assertTrue(addMountTable(addEntry));
-    addEntry = MountTable.newInstance(
-        "/testA/testB/testC/testD", Collections.singletonMap("ns0", "/test"));
-    assertTrue(addMountTable(addEntry));
+    try {
+      // Add mount table entry
+      MountTable addEntry = MountTable.newInstance("/testdir",
+          Collections.singletonMap("ns0", "/testdir"));
+      assertTrue(addMountTable(addEntry));
+      addEntry = MountTable.newInstance("/testdir2",
+          Collections.singletonMap("ns0", "/testdir2"));
+      assertTrue(addMountTable(addEntry));
+      addEntry = MountTable.newInstance("/testdir/subdir",
+          Collections.singletonMap("ns0", "/testdir/subdir"));
+      assertTrue(addMountTable(addEntry));
+      addEntry = MountTable.newInstance("/testdir3/subdir1",
+          Collections.singletonMap("ns0", "/testdir3"));
+      assertTrue(addMountTable(addEntry));
+      addEntry = MountTable.newInstance("/testA/testB/testC/testD",
+          Collections.singletonMap("ns0", "/test"));
+      assertTrue(addMountTable(addEntry));
 
-    // Create test dir in NN
-    final FileSystem nnFs = nnContext.getFileSystem();
-    assertTrue(nnFs.mkdirs(new Path("/newdir")));
+      // Create test dir in NN
+      assertTrue(nnFs0.mkdirs(new Path("/newdir")));
 
-    Map<String, Long> pathModTime = new TreeMap<>();
-    for (String mount : mountTable.getMountPoints("/")) {
-      if (mountTable.getMountPoint("/"+mount) != null) {
-        pathModTime.put(mount, mountTable.getMountPoint("/"+mount)
-            .getDateModified());
-      } else {
-        List<MountTable> entries = mountTable.getMounts("/"+mount);
-        for (MountTable entry : entries) {
-          if (pathModTime.get(mount) == null ||
-              pathModTime.get(mount) < entry.getDateModified()) {
-            pathModTime.put(mount, entry.getDateModified());
+      Map<String, Long> pathModTime = new TreeMap<>();
+      for (String mount : mountTable.getMountPoints("/")) {
+        if (mountTable.getMountPoint("/" + mount) != null) {
+          pathModTime.put(mount,
+              mountTable.getMountPoint("/" + mount).getDateModified());
+        } else {
+          List<MountTable> entries = mountTable.getMounts("/" + mount);
+          for (MountTable entry : entries) {
+            if (pathModTime.get(mount) == null
+                || pathModTime.get(mount) < entry.getDateModified()) {
+              pathModTime.put(mount, entry.getDateModified());
+            }
           }
         }
       }
+      FileStatus[] iterator = nnFs0.listStatus(new Path("/"));
+      for (FileStatus file : iterator) {
+        pathModTime.put(file.getPath().getName(), file.getModificationTime());
+      }
+      // Fetch listing
+      DirectoryListing listing =
+          routerProtocol.getListing("/", HdfsFileStatus.EMPTY_NAME, false);
+      Iterator<String> pathModTimeIterator = pathModTime.keySet().iterator();
+
+      // Match date/time for each path returned
+      for (HdfsFileStatus f : listing.getPartialListing()) {
+        String fileName = pathModTimeIterator.next();
+        String currentFile = f.getFullPath(new Path("/")).getName();
+        Long currentTime = f.getModificationTime();
+        Long expectedTime = pathModTime.get(currentFile);
+
+        assertEquals(currentFile, fileName);
+        assertTrue(currentTime > startTime);
+        assertEquals(currentTime, expectedTime);
+      }
+      // Verify the total number of results found/matched
+      assertEquals(pathModTime.size(), listing.getPartialListing().length);
+    } finally {
+      nnFs0.delete(new Path("/newdir"), true);
     }
-    FileStatus[] iterator = nnFs.listStatus(new Path("/"));
-    for (FileStatus file : iterator) {
-      pathModTime.put(file.getPath().getName(), file.getModificationTime());
+  }
+
+  /**
+   * Verify permission for a mount point when the actual destination is not
+   * present. It returns the permissions of the mount point.
+   */
+  @Test
+  public void testMountTablePermissionsNoDest() throws IOException {
+    MountTable addEntry;
+    addEntry = MountTable.newInstance("/testdir1",
+        Collections.singletonMap("ns0", "/tmp/testdir1"));
+    addEntry.setGroupName("group1");
+    addEntry.setOwnerName("owner1");
+    addEntry.setMode(FsPermission.createImmutable((short) 0775));
+    assertTrue(addMountTable(addEntry));
+    FileStatus[] list = routerContext.getFileSystem().listStatus(new Path("/"));
+    assertEquals("group1", list[0].getGroup());
+    assertEquals("owner1", list[0].getOwner());
+    assertEquals((short) 0775, list[0].getPermission().toShort());
+  }
+
+  /**
+   * Verify permission for a mount point when the actual destination present. It
+   * returns the permissions of the actual destination pointed by the mount
+   * point.
+   */
+  @Test
+  public void testMountTablePermissionsWithDest() throws IOException {
+    try {
+      MountTable addEntry = MountTable.newInstance("/testdir",
+          Collections.singletonMap("ns0", "/tmp/testdir"));
+      assertTrue(addMountTable(addEntry));
+      nnFs0.mkdirs(new Path("/tmp/testdir"));
+      nnFs0.setOwner(new Path("/tmp/testdir"), "Aowner", "Agroup");
+      nnFs0.setPermission(new Path("/tmp/testdir"),
+          FsPermission.createImmutable((short) 775));
+      FileStatus[] list =
+          routerContext.getFileSystem().listStatus(new Path("/"));
+      assertEquals("Agroup", list[0].getGroup());
+      assertEquals("Aowner", list[0].getOwner());
+      assertEquals((short) 775, list[0].getPermission().toShort());
+    } finally {
+      nnFs0.delete(new Path("/tmp"), true);
     }
-    // Fetch listing
-    DirectoryListing listing =
-        routerProtocol.getListing("/", HdfsFileStatus.EMPTY_NAME, false);
-    Iterator<String> pathModTimeIterator = pathModTime.keySet().iterator();
+  }
 
-    // Match date/time for each path returned
-    for(HdfsFileStatus f : listing.getPartialListing()) {
-      String fileName = pathModTimeIterator.next();
-      String currentFile = f.getFullPath(new Path("/")).getName();
-      Long currentTime = f.getModificationTime();
-      Long expectedTime = pathModTime.get(currentFile);
+  /**
+   * Verify permission for a mount point when the multiple destinations are
+   * present with both having same permissions. It returns the same actual
+   * permissions of the actual destinations pointed by the mount point.
+   */
+  @Test
+  public void testMountTablePermissionsMultiDest() throws IOException {
+    try {
+      Map<String, String> destMap = new HashMap<>();
+      destMap.put("ns0", "/tmp/testdir");
+      destMap.put("ns1", "/tmp/testdir01");
+      MountTable addEntry = MountTable.newInstance("/testdir", destMap);
+      assertTrue(addMountTable(addEntry));
+      nnFs0.mkdirs(new Path("/tmp/testdir"));
+      nnFs0.setOwner(new Path("/tmp/testdir"), "Aowner", "Agroup");
+      nnFs0.setPermission(new Path("/tmp/testdir"),
+          FsPermission.createImmutable((short) 775));
+      nnFs1.mkdirs(new Path("/tmp/testdir01"));
+      nnFs1.setOwner(new Path("/tmp/testdir01"), "Aowner", "Agroup");
+      nnFs1.setPermission(new Path("/tmp/testdir01"),
+          FsPermission.createImmutable((short) 775));
+      FileStatus[] list =
+          routerContext.getFileSystem().listStatus(new Path("/"));
+      assertEquals("Agroup", list[0].getGroup());
+      assertEquals("Aowner", list[0].getOwner());
+      assertEquals((short) 775, list[0].getPermission().toShort());
+    } finally {
+      nnFs0.delete(new Path("/tmp"), true);
+      nnFs1.delete(new Path("/tmp"), true);
+    }
+  }
 
-      assertEquals(currentFile, fileName);
-      assertTrue(currentTime > startTime);
-      assertEquals(currentTime, expectedTime);
+  /**
+   * Verify permission for a mount point when the multiple destinations are
+   * present with both having different permissions. It returns the actual
+   * permissions of either of the actual destinations pointed by the mount
+   * point.
+   */
+  @Test
+  public void testMountTablePermissionsMultiDestDifferentPerm()
+      throws IOException {
+    try {
+      Map<String, String> destMap = new HashMap<>();
+      destMap.put("ns0", "/tmp/testdir");
+      destMap.put("ns1", "/tmp/testdir01");
+      MountTable addEntry = MountTable.newInstance("/testdir", destMap);
+      assertTrue(addMountTable(addEntry));
+      nnFs0.mkdirs(new Path("/tmp/testdir"));
+      nnFs0.setOwner(new Path("/tmp/testdir"), "Aowner", "Agroup");
+      nnFs0.setPermission(new Path("/tmp/testdir"),
+          FsPermission.createImmutable((short) 775));
+      nnFs1.mkdirs(new Path("/tmp/testdir01"));
+      nnFs1.setOwner(new Path("/tmp/testdir01"), "Aowner01", "Agroup01");
+      nnFs1.setPermission(new Path("/tmp/testdir01"),
+          FsPermission.createImmutable((short) 755));
+      FileStatus[] list =
+          routerContext.getFileSystem().listStatus(new Path("/"));
+      assertTrue("Agroup".equals(list[0].getGroup())
+          || "Agroup01".equals(list[0].getGroup()));
+      assertTrue("Aowner".equals(list[0].getOwner())
+          || "Aowner01".equals(list[0].getOwner()));
+      assertTrue(((short) 775) == list[0].getPermission().toShort()
+          || ((short) 755) == list[0].getPermission().toShort());
+    } finally {
+      nnFs0.delete(new Path("/tmp"), true);
+      nnFs1.delete(new Path("/tmp"), true);
     }
-    // Verify the total number of results found/matched
-    assertEquals(pathModTime.size(), listing.getPartialListing().length);
   }
 
   /**
-   * Verify that the file listing contains correct permission.
+   * Validate whether mount point name gets resolved or not. On successful
+   * resolution the details returned would be the ones actually set on the mount
+   * point.
    */
   @Test
-  public void testMountTablePermissions() throws Exception {
-    // Add mount table entries
-    MountTable addEntry = MountTable.newInstance(
-        "/testdir1", Collections.singletonMap("ns0", "/testdir1"));
+  public void testMountPointResolved() throws IOException {
+    MountTable addEntry = MountTable.newInstance("/testdir",
+        Collections.singletonMap("ns0", "/tmp/testdir"));
     addEntry.setGroupName("group1");
     addEntry.setOwnerName("owner1");
-    addEntry.setMode(FsPermission.createImmutable((short)0775));
-    assertTrue(addMountTable(addEntry));
-    addEntry = MountTable.newInstance(
-        "/testdir2", Collections.singletonMap("ns0", "/testdir2"));
-    addEntry.setGroupName("group2");
-    addEntry.setOwnerName("owner2");
-    addEntry.setMode(FsPermission.createImmutable((short)0755));
     assertTrue(addMountTable(addEntry));
+    HdfsFileStatus finfo = routerProtocol.getFileInfo("/testdir");
+    FileStatus[] finfo1 =
+        routerContext.getFileSystem().listStatus(new Path("/"));
+    assertEquals("owner1", finfo.getOwner());
+    assertEquals("owner1", finfo1[0].getOwner());
+    assertEquals("group1", finfo.getGroup());
+    assertEquals("group1", finfo1[0].getGroup());
+  }
 
-    HdfsFileStatus fs = routerProtocol.getFileInfo("/testdir1");
-    assertEquals("group1", fs.getGroup());
-    assertEquals("owner1", fs.getOwner());
-    assertEquals((short) 0775, fs.getPermission().toShort());
+  /**
+   * Validate the number of children for the mount point.It must be equal to the
+   * number of children of the destination pointed by the mount point.
+   */
+  @Test
+  public void testMountPointChildren() throws IOException {
+    try {
+      MountTable addEntry = MountTable.newInstance("/testdir",
+          Collections.singletonMap("ns0", "/tmp/testdir"));
+      assertTrue(addMountTable(addEntry));
+      nnFs0.mkdirs(new Path("/tmp/testdir"));
+      nnFs0.mkdirs(new Path("/tmp/testdir/1"));
+      nnFs0.mkdirs(new Path("/tmp/testdir/2"));
+      FileStatus[] finfo1 =
+          routerContext.getFileSystem().listStatus(new Path("/"));
+      assertEquals(2, ((HdfsFileStatus) finfo1[0]).getChildrenNum());
+    } finally {
+      nnFs0.delete(new Path("/tmp"), true);
+    }
+  }
 
-    fs = routerProtocol.getFileInfo("/testdir2");
-    assertEquals("group2", fs.getGroup());
-    assertEquals("owner2", fs.getOwner());
-    assertEquals((short) 0755, fs.getPermission().toShort());
+  /**
+   * Validate the number of children for the mount point pointing to multiple
+   * destinations.It must be equal to the sum of number of children of the
+   * destinations pointed by the mount point.
+   */
+  @Test
+  public void testMountPointChildrenMultiDest() throws IOException {
+    try {
+      Map<String, String> destMap = new HashMap<>();
+      destMap.put("ns0", "/tmp/testdir");
+      destMap.put("ns1", "/tmp/testdir01");
+      MountTable addEntry = MountTable.newInstance("/testdir", destMap);
+      assertTrue(addMountTable(addEntry));
+      nnFs0.mkdirs(new Path("/tmp/testdir"));
+      nnFs0.mkdirs(new Path("/tmp/testdir"));
+      nnFs1.mkdirs(new Path("/tmp/testdir01"));
+      nnFs0.mkdirs(new Path("/tmp/testdir/1"));
+      nnFs1.mkdirs(new Path("/tmp/testdir01/1"));
+      FileStatus[] finfo1 =
+          routerContext.getFileSystem().listStatus(new Path("/"));
+      assertEquals(2, ((HdfsFileStatus) finfo1[0]).getChildrenNum());
+    } finally {
+      nnFs0.delete(new Path("/tmp"), true);
+      nnFs0.delete(new Path("/tmp"), true);
+    }
   }
 }
\ No newline at end of file


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