Sfoglia il codice sorgente

HDFS-17749. [ARR] Fix wrong results due to the wrong usage of asyncComplete in getListing. (#7466). Contributed by hfutatzhanghb.

Reviewed-by: Jian Zhang <keepromise@apache.org>
hfutatzhanghb 1 mese fa
parent
commit
fb30082948

+ 2 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterClientProtocol.java

@@ -2178,12 +2178,12 @@ public class RouterClientProtocol implements ClientProtocol {
   protected HdfsFileStatus getFileInfoAll(final List<RemoteLocation> locations,
       final RemoteMethod method, long timeOutMs) throws IOException {
 
-    // Get the file info from everybody
+    // Get the file info from everybody.
     Map<RemoteLocation, HdfsFileStatus> results =
         rpcClient.invokeConcurrent(locations, method, false, false, timeOutMs,
             HdfsFileStatus.class);
     int children = 0;
-    // We return the first file
+    // We return the first file.
     HdfsFileStatus dirStatus = null;
     for (RemoteLocation loc : locations) {
       HdfsFileStatus fileStatus = results.get(loc);

+ 87 - 46
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/async/RouterAsyncClientProtocol.java

@@ -355,7 +355,6 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
       return rpcClient.invokeAll(locations, method);
     }
 
-    asyncComplete(false);
     if (locations.size() > 1) {
       // Check if this directory already exists
       asyncTry(() -> {
@@ -368,32 +367,41 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
           return false;
         });
       });
-      asyncCatch((ret, ex) -> {
+      asyncCatch((ret, ioe) -> {
         // Can't query if this file exists or not.
         LOG.error("Error getting file info for {} while proxying mkdirs: {}",
-            src, ex.getMessage());
+            src, ioe.getMessage());
         return false;
       }, IOException.class);
-    }
 
-    final RemoteLocation firstLocation = locations.get(0);
-    asyncApply((AsyncApplyFunction<Boolean, Boolean>) success -> {
-      if (success) {
-        asyncComplete(true);
-        return;
-      }
+      asyncApply((AsyncApplyFunction<Boolean, Boolean>)ret -> {
+        if (!ret) {
+          final RemoteLocation firstLocation = locations.get(0);
+          asyncTry(() -> {
+            rpcClient.invokeSingle(firstLocation, method, Boolean.class);
+          });
+          asyncCatch((AsyncCatchFunction<Object, IOException>) (o, ioe) -> {
+            final List<RemoteLocation> newLocations = checkFaultTolerantRetry(
+                method, src, ioe, firstLocation, locations);
+            rpcClient.invokeSequential(
+                newLocations, method, Boolean.class, Boolean.TRUE);
+          }, IOException.class);
+        } else {
+          asyncComplete(ret);
+        }
+      });
+    } else {
+      final RemoteLocation firstLocation = locations.get(0);
       asyncTry(() -> {
         rpcClient.invokeSingle(firstLocation, method, Boolean.class);
       });
-
       asyncCatch((AsyncCatchFunction<Object, IOException>) (o, ioe) -> {
         final List<RemoteLocation> newLocations = checkFaultTolerantRetry(
             method, src, ioe, firstLocation, locations);
         rpcClient.invokeSequential(
             newLocations, method, Boolean.class, Boolean.TRUE);
       }, IOException.class);
-    });
-
+    }
     return asyncReturn(Boolean.class);
   }
 
@@ -487,6 +495,7 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
               return null;
             });
           });
+          boolean finalNamenodeListingExists = namenodeListingExists;
           asyncApply(o -> {
             // Update the remaining count to include left mount points
             if (nnListing.size() > 0) {
@@ -499,10 +508,12 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
                 }
               }
             }
-            return null;
+            return finalNamenodeListingExists;
           });
+        } else {
+          asyncComplete(namenodeListingExists);
         }
-        asyncComplete(namenodeListingExists);
+
         asyncApply((ApplyFunction<Boolean, Object>) exists -> {
           if (!exists && nnListing.size() == 0 && children == null) {
             // NN returns a null object if the directory cannot be found and has no
@@ -547,7 +558,6 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
     return asyncReturn(List.class);
   }
 
-
   @Override
   public HdfsFileStatus getFileInfo(String src) throws IOException {
     rpcServer.checkOperation(NameNode.OperationCategory.READ);
@@ -570,7 +580,7 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
           || e instanceof RouterResolveException) {
         noLocationException[0] = e;
       }
-      throw e;
+      return null;
     }, IOException.class);
 
     asyncApply((AsyncApplyFunction<HdfsFileStatus, Object>) ret -> {
@@ -588,7 +598,11 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
           // The src is a mount point, but there are no files or directories
           getMountPointStatus(src, 0, 0, false);
         } else {
+          if (noLocationException[0] != null) {
+            throw noLocationException[0];
+          }
           asyncComplete(null);
+          return;
         }
         asyncApply((ApplyFunction<HdfsFileStatus, HdfsFileStatus>) result -> {
           // Can't find mount point for path and the path didn't contain any sub monit points,
@@ -596,7 +610,6 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
           if (result == null && noLocationException[0] != null) {
             throw noLocationException[0];
           }
-
           return result;
         });
       } else {
@@ -645,7 +658,14 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
     final int[] childrenNums = new int[]{childrenNum};
     final EnumSet<HdfsFileStatus.Flags>[] flags =
         new EnumSet[]{EnumSet.noneOf(HdfsFileStatus.Flags.class)};
-    asyncComplete(null);
+    long inodeId = 0;
+    HdfsFileStatus.Builder builder = new HdfsFileStatus.Builder();
+    if (setPath) {
+      Path path = new Path(name);
+      String nameStr = path.getName();
+      builder.path(DFSUtil.string2Bytes(nameStr));
+    }
+
     if (getSubclusterResolver() instanceof MountTableResolver) {
       asyncTry(() -> {
         String mName = name.startsWith("/") ? name : "/" + name;
@@ -670,13 +690,45 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
                   .getFlags(fInfo.isEncrypted(), fInfo.isErasureCoded(),
                       fInfo.isSnapshotEnabled(), fInfo.hasAcl());
             }
-            return fInfo;
+            return builder.isdir(true)
+                .mtime(modTime)
+                .atime(accessTime)
+                .perm(permission[0])
+                .owner(owner[0])
+                .group(group[0])
+                .symlink(new byte[0])
+                .fileId(inodeId)
+                .children(childrenNums[0])
+                .flags(flags[0])
+                .build();
           });
+        } else {
+          asyncComplete(builder.isdir(true)
+              .mtime(modTime)
+              .atime(accessTime)
+              .perm(permission[0])
+              .owner(owner[0])
+              .group(group[0])
+              .symlink(new byte[0])
+              .fileId(inodeId)
+              .children(childrenNums[0])
+              .flags(flags[0])
+              .build());
         }
       });
       asyncCatch((CatchFunction<HdfsFileStatus, IOException>) (status, e) -> {
         LOG.error("Cannot get mount point: {}", e.getMessage());
-        return status;
+        return builder.isdir(true)
+            .mtime(modTime)
+            .atime(accessTime)
+            .perm(permission[0])
+            .owner(owner[0])
+            .group(group[0])
+            .symlink(new byte[0])
+            .fileId(inodeId)
+            .children(childrenNums[0])
+            .flags(flags[0])
+            .build();
       }, IOException.class);
     } else {
       try {
@@ -690,44 +742,33 @@ public class RouterAsyncClientProtocol extends RouterClientProtocol {
         } else {
           LOG.debug(msg);
         }
+      } finally {
+        asyncComplete(builder.isdir(true)
+            .mtime(modTime)
+            .atime(accessTime)
+            .perm(permission[0])
+            .owner(owner[0])
+            .group(group[0])
+            .symlink(new byte[0])
+            .fileId(inodeId)
+            .children(childrenNums[0])
+            .flags(flags[0])
+            .build());
       }
     }
-    long inodeId = 0;
-    HdfsFileStatus.Builder builder = new HdfsFileStatus.Builder();
-    asyncApply((ApplyFunction<HdfsFileStatus, HdfsFileStatus>) status -> {
-      if (setPath) {
-        Path path = new Path(name);
-        String nameStr = path.getName();
-        builder.path(DFSUtil.string2Bytes(nameStr));
-      }
-
-      return builder.isdir(true)
-          .mtime(modTime)
-          .atime(accessTime)
-          .perm(permission[0])
-          .owner(owner[0])
-          .group(group[0])
-          .symlink(new byte[0])
-          .fileId(inodeId)
-          .children(childrenNums[0])
-          .flags(flags[0])
-          .build();
-    });
     return asyncReturn(HdfsFileStatus.class);
   }
 
   @Override
   protected HdfsFileStatus getFileInfoAll(final List<RemoteLocation> locations,
       final RemoteMethod method, long timeOutMs) throws IOException {
-
-    asyncComplete(null);
-    // Get the file info from everybody
+    // Get the file info from everybody.
     rpcClient.invokeConcurrent(locations, method, false, false, timeOutMs,
         HdfsFileStatus.class);
     asyncApply(res -> {
       Map<RemoteLocation, HdfsFileStatus> results = (Map<RemoteLocation, HdfsFileStatus>) res;
       int children = 0;
-      // We return the first file
+      // We return the first file.
       HdfsFileStatus dirStatus = null;
       for (RemoteLocation loc : locations) {
         HdfsFileStatus fileStatus = results.get(loc);

+ 36 - 35
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterMountTable.java

@@ -68,24 +68,25 @@ import org.junit.Test;
 /**
  * Test a router end-to-end including the MountTable.
  */
+@SuppressWarnings("checkstyle:VisibilityModifier")
 public class TestRouterMountTable {
 
-  private static StateStoreDFSCluster cluster;
-  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;
-  private static FileSystem routerFs;
+  protected static StateStoreDFSCluster cluster;
+  protected static NamenodeContext nnContext0;
+  protected static NamenodeContext nnContext1;
+  protected static RouterContext routerContext;
+  protected static MountTableResolver mountTable;
+  protected static ClientProtocol routerProtocol;
+  protected static long startTime;
+  protected static FileSystem nnFs0;
+  protected static FileSystem nnFs1;
+  protected static FileSystem routerFs;
 
   @BeforeClass
   public static void globalSetUp() throws Exception {
     startTime = Time.now();
 
-    // Build and start a federated cluster
+    // Build and start a federated cluster.
     cluster = new StateStoreDFSCluster(false, 2);
     Configuration conf = new RouterConfigBuilder()
         .stateStore()
@@ -98,7 +99,7 @@ public class TestRouterMountTable {
     cluster.startRouters();
     cluster.waitClusterUp();
 
-    // Get the end points
+    // Get the end points.
     nnContext0 = cluster.getNamenode("ns0", null);
     nnContext1 = cluster.getNamenode("ns1", null);
     nnFs0 = nnContext0.getFileSystem();
@@ -138,18 +139,18 @@ public class TestRouterMountTable {
   @Test
   public void testReadOnly() throws Exception {
 
-    // Add a read only entry
+    // Add a read only entry.
     MountTable readOnlyEntry = MountTable.newInstance(
         "/readonly", Collections.singletonMap("ns0", "/testdir"));
     readOnlyEntry.setReadOnly(true);
     assertTrue(addMountTable(readOnlyEntry));
 
-    // Add a regular entry
+    // Add a regular entry.
     MountTable regularEntry = MountTable.newInstance(
         "/regular", Collections.singletonMap("ns0", "/testdir"));
     assertTrue(addMountTable(regularEntry));
 
-    // Create a folder which should show in all locations
+    // Create a folder which should show in all locations.
     assertTrue(routerFs.mkdirs(new Path("/regular/newdir")));
 
     FileStatus dirStatusNn =
@@ -162,7 +163,7 @@ public class TestRouterMountTable {
         routerFs.getFileStatus(new Path("/readonly/newdir"));
     assertTrue(dirStatusReadOnly.isDirectory());
 
-    // It should fail writing into a read only path
+    // It should fail writing into a read only path.
     try {
       routerFs.mkdirs(new Path("/readonly/newdirfail"));
       fail("We should not be able to write into a read only mount point");
@@ -187,7 +188,7 @@ public class TestRouterMountTable {
     AddMountTableEntryResponse addResponse =
         mountTableManager.addMountTableEntry(addRequest);
 
-    // Reload the Router cache
+    // Reload the Router cache.
     mountTable.loadCache(true);
 
     return addResponse.getStatus();
@@ -207,7 +208,7 @@ public class TestRouterMountTable {
     UpdateMountTableEntryResponse updateResponse =
         mountTableManager.updateMountTableEntry(updateRequest);
 
-    // Reload the Router cache
+    // Reload the Router cache.
     mountTable.loadCache(true);
 
     return updateResponse.getStatus();
@@ -219,7 +220,7 @@ public class TestRouterMountTable {
    */
   @Test
   public void testMountPointLimit() throws Exception {
-    // Add mount table entry
+    // Add mount table entry.
     MountTable addEntry = MountTable.newInstance("/testdir-shortlength",
         Collections.singletonMap("ns0", "/testdir-shortlength"));
     assertTrue(addMountTable(addEntry));
@@ -248,7 +249,7 @@ public class TestRouterMountTable {
   @Test
   public void testListFilesTime() throws Exception {
     try {
-      // Add mount table entry
+      // Add mount table entry.
       MountTable addEntry = MountTable.newInstance("/testdir",
           Collections.singletonMap("ns0", "/testdir"));
       assertTrue(addMountTable(addEntry));
@@ -265,7 +266,7 @@ public class TestRouterMountTable {
           Collections.singletonMap("ns0", "/test"));
       assertTrue(addMountTable(addEntry));
 
-      // Create test dir in NN
+      // Create test dir in NN.
       assertTrue(nnFs0.mkdirs(new Path("/newdir")));
 
       Map<String, Long> pathModTime = new TreeMap<>();
@@ -287,12 +288,12 @@ public class TestRouterMountTable {
       for (FileStatus file : iterator) {
         pathModTime.put(file.getPath().getName(), file.getModificationTime());
       }
-      // Fetch listing
+      // Fetch listing.
       DirectoryListing listing =
           routerProtocol.getListing("/", HdfsFileStatus.EMPTY_NAME, false);
       Iterator<String> pathModTimeIterator = pathModTime.keySet().iterator();
 
-      // Match date/time for each path returned
+      // Match date/time for each path returned.
       for (HdfsFileStatus f : listing.getPartialListing()) {
         String fileName = pathModTimeIterator.next();
         String currentFile = f.getFullPath(new Path("/")).getName();
@@ -303,7 +304,7 @@ public class TestRouterMountTable {
         assertTrue(currentTime > startTime);
         assertEquals(currentTime, expectedTime);
       }
-      // Verify the total number of results found/matched
+      // Verify the total number of results found/matched.
       assertEquals(pathModTime.size(), listing.getPartialListing().length);
     } finally {
       nnFs0.delete(new Path("/newdir"), true);
@@ -645,7 +646,7 @@ public class TestRouterMountTable {
   @Test
   public void testGetListingWithTrailingSlash() throws IOException {
     try {
-      // Add mount table entry
+      // Add mount table entry.
       MountTable addEntry = MountTable.newInstance("/testlist",
           Collections.singletonMap("ns0", "/testlist"));
       assertTrue(addMountTable(addEntry));
@@ -658,11 +659,11 @@ public class TestRouterMountTable {
 
       nnFs0.mkdirs(new Path("/testlist/tmp0"));
       nnFs1.mkdirs(new Path("/testlist/tmp1"));
-      // Fetch listing
+      // Fetch listing.
       DirectoryListing list = routerProtocol.getListing(
           "/testlist/", HdfsFileStatus.EMPTY_NAME, false);
       HdfsFileStatus[] statuses = list.getPartialListing();
-      // should return tmp0 and tmp1
+      // Should return tmp0 and tmp1.
       assertEquals(2, statuses.length);
     } finally {
       nnFs0.delete(new Path("/testlist/tmp0"), true);
@@ -673,7 +674,7 @@ public class TestRouterMountTable {
   @Test
   public void testGetFileInfoWithMountPoint() throws IOException {
     try {
-      // Add mount table entry
+      // Add mount table entry.
       MountTable addEntry = MountTable.newInstance("/testgetfileinfo/ns1/dir",
           Collections.singletonMap("ns1", "/testgetfileinfo/ns1/dir"));
       assertTrue(addMountTable(addEntry));
@@ -739,7 +740,7 @@ public class TestRouterMountTable {
       nnFs0.mkdirs(new Path("/testrename1/sub/sub"));
       nnFs0.mkdirs(new Path("/testrename2"));
 
-      // Success: rename a directory to a mount point
+      // Success: rename a directory to a mount point.
       assertTrue(nnFs0.exists(new Path("/testrename1/sub/sub")));
       assertFalse(nnFs0.exists(new Path("/testrename2/sub")));
       assertTrue(routerFs.rename(new Path("/testrename1/sub/sub"),
@@ -747,19 +748,19 @@ public class TestRouterMountTable {
       assertFalse(nnFs0.exists(new Path("/testrename1/sub/sub")));
       assertTrue(nnFs0.exists(new Path("/testrename2/sub")));
 
-      // Fail: the target already exists
+      // Fail: the target already exists.
       nnFs0.mkdirs(new Path("/testrename1/sub/sub"));
       assertFalse(routerFs.rename(new Path("/testrename1/sub/sub"),
               new Path("/testrename2")));
 
-      // Fail: The src is a mount point
+      // Fail: The src is a mount point.
       LambdaTestUtils.intercept(AccessControlException.class,
           "The operation is not allowed because the path: "
               + "/testrename1/sub is a mount point",
           () -> routerFs.rename(new Path("/testrename1/sub"),
               new Path("/testrename2/sub")));
 
-      // Fail: There is a mount point under the src
+      // Fail: There is a mount point under the src.
       LambdaTestUtils.intercept(AccessControlException.class,
           "The operation is not allowed because there are mount points: "
               + "sub under the path: /testrename1",
@@ -790,7 +791,7 @@ public class TestRouterMountTable {
   @Test
   public void testGetEnclosingRoot() throws Exception {
 
-    // Add a read only entry
+    // Add a read only entry.
     MountTable readOnlyEntry = MountTable.newInstance(
         "/readonly", Collections.singletonMap("ns0", "/testdir"));
     readOnlyEntry.setReadOnly(true);
@@ -801,13 +802,13 @@ public class TestRouterMountTable {
     assertEquals(routerFs.getEnclosingRoot(new Path("/regular")),
         routerFs.getEnclosingRoot(routerFs.getEnclosingRoot(new Path("/regular"))));
 
-    // Add a regular entry
+    // Add a regular entry.
     MountTable regularEntry = MountTable.newInstance(
         "/regular", Collections.singletonMap("ns0", "/testdir"));
     assertTrue(addMountTable(regularEntry));
     assertEquals(routerFs.getEnclosingRoot(new Path("/regular")), new Path("/regular"));
 
-    // path does not need to exist
+    // Path does not need to exist.
     assertEquals(routerFs.getEnclosingRoot(new Path("/regular/pathDNE")), new Path("/regular"));
 
   }

+ 11 - 105
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/async/TestRouterAsyncMountTable.java

@@ -18,50 +18,27 @@
 package org.apache.hadoop.hdfs.server.federation.router.async;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.server.federation.MiniRouterDFSCluster;
 import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
 import org.apache.hadoop.hdfs.server.federation.StateStoreDFSCluster;
-import org.apache.hadoop.hdfs.server.federation.resolver.MountTableManager;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
 import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
 import org.apache.hadoop.hdfs.server.federation.router.Router;
-import org.apache.hadoop.hdfs.server.federation.router.RouterClient;
-import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryRequest;
-import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntryResponse;
-import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesRequest;
-import org.apache.hadoop.hdfs.server.federation.store.protocol.GetMountTableEntriesResponse;
-import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableEntryRequest;
-import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
-import org.apache.hadoop.test.LambdaTestUtils;
-import org.junit.After;
-import org.junit.AfterClass;
+import org.apache.hadoop.hdfs.server.federation.router.TestRouterMountTable;
+import org.apache.hadoop.util.Time;
 import org.junit.BeforeClass;
-import org.junit.Test;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.util.Collections;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
 /**
  * Test a router end-to-end including the MountTable using async rpc.
  */
-public class TestRouterAsyncMountTable {
+public class TestRouterAsyncMountTable extends TestRouterMountTable {
   public static final Logger LOG = LoggerFactory.getLogger(TestRouterAsyncMountTable.class);
 
-  private static StateStoreDFSCluster cluster;
-  private static MiniRouterDFSCluster.RouterContext routerContext;
-  private static MountTableResolver mountTable;
-  private static FileSystem routerFs;
-
   @BeforeClass
   public static void globalSetUp() throws Exception {
+    startTime = Time.now();
+
     // Build and start a federated cluster.
     cluster = new StateStoreDFSCluster(false, 2);
     Configuration conf = new RouterConfigBuilder()
@@ -76,86 +53,15 @@ public class TestRouterAsyncMountTable {
     cluster.startRouters();
     cluster.waitClusterUp();
 
+    // Get the end points.
+    nnContext0 = cluster.getNamenode("ns0", null);
+    nnContext1 = cluster.getNamenode("ns1", null);
+    nnFs0 = nnContext0.getFileSystem();
+    nnFs1 = nnContext1.getFileSystem();
     routerContext = cluster.getRandomRouter();
     routerFs = routerContext.getFileSystem();
     Router router = routerContext.getRouter();
+    routerProtocol = routerContext.getClient().getNamenode();
     mountTable = (MountTableResolver) router.getSubclusterResolver();
   }
-
-  @AfterClass
-  public static void tearDown() {
-    if (cluster != null) {
-      cluster.stopRouter(routerContext);
-      cluster.shutdown();
-      cluster = null;
-    }
-  }
-
-  @After
-  public void clearMountTable() throws IOException {
-    RouterClient client = routerContext.getAdminClient();
-    MountTableManager mountTableManager = client.getMountTableManager();
-    GetMountTableEntriesRequest req1 =
-        GetMountTableEntriesRequest.newInstance("/");
-    GetMountTableEntriesResponse response =
-        mountTableManager.getMountTableEntries(req1);
-    for (MountTable entry : response.getEntries()) {
-      RemoveMountTableEntryRequest req2 =
-          RemoveMountTableEntryRequest.newInstance(entry.getSourcePath());
-      mountTableManager.removeMountTableEntry(req2);
-    }
-    mountTable.setDefaultNSEnable(true);
-  }
-
-  /**
-   * Add a mount table entry to the mount table through the admin API.
-   * @param entry Mount table entry to add.
-   * @return If it was succesfully added.
-   * @throws IOException Problems adding entries.
-   */
-  private boolean addMountTable(final MountTable entry) throws IOException {
-    RouterClient client = routerContext.getAdminClient();
-    MountTableManager mountTableManager = client.getMountTableManager();
-    AddMountTableEntryRequest addRequest =
-        AddMountTableEntryRequest.newInstance(entry);
-    AddMountTableEntryResponse addResponse =
-        mountTableManager.addMountTableEntry(addRequest);
-
-    // Reload the Router cache.
-    mountTable.loadCache(true);
-
-    return addResponse.getStatus();
-  }
-
-  @Test
-  public void testGetEnclosingRoot() throws Exception {
-    // Add a read only entry.
-    MountTable readOnlyEntry = MountTable.newInstance(
-        "/readonly", Collections.singletonMap("ns0", "/testdir"));
-    readOnlyEntry.setReadOnly(true);
-    assertTrue(addMountTable(readOnlyEntry));
-    assertEquals(routerFs.getEnclosingRoot(new Path("/readonly")), new Path("/readonly"));
-
-    assertEquals(routerFs.getEnclosingRoot(new Path("/regular")), new Path("/"));
-    assertEquals(routerFs.getEnclosingRoot(new Path("/regular")),
-        routerFs.getEnclosingRoot(routerFs.getEnclosingRoot(new Path("/regular"))));
-
-    // Add a regular entry.
-    MountTable regularEntry = MountTable.newInstance(
-        "/regular", Collections.singletonMap("ns0", "/testdir"));
-    assertTrue(addMountTable(regularEntry));
-    assertEquals(routerFs.getEnclosingRoot(new Path("/regular")), new Path("/regular"));
-
-    // Path does not need to exist.
-    assertEquals(routerFs.getEnclosingRoot(new Path("/regular/pathDNE")), new Path("/regular"));
-  }
-
-  @Test
-  public void testListNonExistPath() throws Exception {
-    mountTable.setDefaultNSEnable(false);
-    LambdaTestUtils.intercept(FileNotFoundException.class,
-        "File /base does not exist.",
-        "Expect FileNotFoundException.",
-        () -> routerFs.listStatus(new Path("/base")));
-  }
 }