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 ay...@apache.org on 2019/10/16 14:11:34 UTC

[hadoop] branch trunk updated: HDFS-14739. RBF: LS command for mount point shows wrong owner and permission information. Contributed by Jinglun.

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

ayushsaxena pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/hadoop.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 375224e  HDFS-14739. RBF: LS command for mount point shows wrong owner and permission information. Contributed by Jinglun.
375224e is described below

commit 375224edebb1c937afe4bbea8fe884499ca8ece5
Author: Ayush Saxena <ay...@apache.org>
AuthorDate: Wed Oct 16 19:26:01 2019 +0530

    HDFS-14739. RBF: LS command for mount point shows wrong owner and permission information. Contributed by Jinglun.
---
 .../federation/resolver/MountTableResolver.java    |  4 +-
 .../resolver/RouterResolveException.java           | 32 +++++++++++++
 .../federation/router/RouterClientProtocol.java    | 42 ++++++++++++-----
 .../federation/router/TestDisableNameservices.java | 23 +++++-----
 .../federation/router/TestRouterMountTable.java    | 52 +++++++++++++++++++++-
 5 files changed, 125 insertions(+), 28 deletions(-)

diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
index c72afc5..6becedc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/MountTableResolver.java
@@ -423,8 +423,8 @@ public class MountTableResolver
     } else {
       // Not found, use default location
       if (!defaultNSEnable) {
-        throw new IOException("Cannot find locations for " + path + ", " +
-            "because the default nameservice is disabled to read or write");
+        throw new RouterResolveException("Cannot find locations for " + path
+            + ", because the default nameservice is disabled to read or write");
       }
       RemoteLocation remoteLocation =
           new RemoteLocation(defaultNameService, path, path);
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RouterResolveException.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RouterResolveException.java
new file mode 100644
index 0000000..cee466d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/resolver/RouterResolveException.java
@@ -0,0 +1,32 @@
+/**
+ * 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.hdfs.server.federation.resolver;
+
+import java.io.IOException;
+
+/**
+ * Thrown by FileSubclusterResolver when a path can't be resolved.
+ */
+public class RouterResolveException extends IOException {
+  /** For java.io.Serializable. */
+  private static final long serialVersionUID = 1L;
+
+  public RouterResolveException(String msg) {
+    super(msg);
+  }
+}
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 b5b19a3..0168e22 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
@@ -78,6 +78,7 @@ import org.apache.hadoop.hdfs.server.federation.resolver.FederationNamespaceInfo
 import org.apache.hadoop.hdfs.server.federation.resolver.FileSubclusterResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.MountTableResolver;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
+import org.apache.hadoop.hdfs.server.federation.resolver.RouterResolveException;
 import org.apache.hadoop.hdfs.server.federation.router.security.RouterSecurityManager;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -745,16 +746,8 @@ public class RouterClientProtocol implements ClientProtocol {
       boolean needLocation) throws IOException {
     rpcServer.checkOperation(NameNode.OperationCategory.READ);
 
-    // Locate the dir and fetch the listing
-    final List<RemoteLocation> locations =
-        rpcServer.getLocationsForPath(src, false, false);
-    RemoteMethod method = new RemoteMethod("getListing",
-        new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
-        new RemoteParam(), startAfter, needLocation);
-    final List<RemoteResult<RemoteLocation, DirectoryListing>> listings =
-        rpcClient.invokeConcurrent(
-            locations, method, false, -1, DirectoryListing.class);
-
+    List<RemoteResult<RemoteLocation, DirectoryListing>> listings =
+        getListingInt(src, startAfter, needLocation);
     Map<String, HdfsFileStatus> nnListing = new TreeMap<>();
     int totalRemainingEntries = 0;
     int remainingEntries = 0;
@@ -818,7 +811,9 @@ public class RouterClientProtocol implements ClientProtocol {
         if (dates != null && dates.containsKey(child)) {
           date = dates.get(child);
         }
-        HdfsFileStatus dirStatus = getMountPointStatus(child, 0, date);
+        Path childPath = new Path(src, child);
+        HdfsFileStatus dirStatus =
+            getMountPointStatus(childPath.toString(), 0, date);
 
         // This may overwrite existing listing entries with the mount point
         // TODO don't add if already there?
@@ -2043,6 +2038,31 @@ public class RouterClientProtocol implements ClientProtocol {
   }
 
   /**
+   * Get listing on remote locations.
+   */
+  private List<RemoteResult<RemoteLocation, DirectoryListing>> getListingInt(
+      String src, byte[] startAfter, boolean needLocation) throws IOException {
+    try {
+      List<RemoteLocation> locations =
+          rpcServer.getLocationsForPath(src, false, false);
+      // Locate the dir and fetch the listing.
+      if (locations.isEmpty()) {
+        return new ArrayList<>();
+      }
+      RemoteMethod method = new RemoteMethod("getListing",
+          new Class<?>[] {String.class, startAfter.getClass(), boolean.class},
+          new RemoteParam(), startAfter, needLocation);
+      List<RemoteResult<RemoteLocation, DirectoryListing>> listings = rpcClient
+          .invokeConcurrent(locations, method, false, -1,
+              DirectoryListing.class);
+      return listings;
+    } catch (RouterResolveException e) {
+      LOG.debug("Cannot get locations for {}, {}.", src, e.getMessage());
+      return new ArrayList<>();
+    }
+  }
+
+  /**
    * Checks if the path is a directory and is supposed to be present in all
    * subclusters.
    * @param src the source path
diff --git a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableNameservices.java b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableNameservices.java
index 3d803d6..ae04150 100644
--- a/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableNameservices.java
+++ b/hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableNameservices.java
@@ -21,7 +21,6 @@ import static org.apache.hadoop.hdfs.server.federation.FederationTestUtils.simul
 import static org.apache.hadoop.util.Time.monotonicNow;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
 
 import java.io.IOException;
 import java.util.Iterator;
@@ -50,7 +49,6 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.AddMountTableEntr
 import org.apache.hadoop.hdfs.server.federation.store.protocol.DisableNameserviceRequest;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
-import org.apache.hadoop.test.GenericTestUtils;
 import org.codehaus.jettison.json.JSONObject;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -128,6 +126,7 @@ public class TestDisableNameservices {
     // Add a folder to each namespace
     NamenodeContext nn0 = cluster.getNamenode("ns0", null);
     nn0.getFileSystem().mkdirs(new Path("/dirns0/0"));
+    nn0.getFileSystem().mkdirs(new Path("/dir-ns"));
     NamenodeContext nn1 = cluster.getNamenode("ns1", null);
     nn1.getFileSystem().mkdirs(new Path("/dirns1/1"));
   }
@@ -167,9 +166,10 @@ public class TestDisableNameservices {
     // Return the results from all subclusters even if slow
     FileSystem routerFs = routerContext.getFileSystem();
     FileStatus[] filesStatus = routerFs.listStatus(new Path("/"));
-    assertEquals(2, filesStatus.length);
-    assertEquals("dirns0", filesStatus[0].getPath().getName());
-    assertEquals("dirns1", filesStatus[1].getPath().getName());
+    assertEquals(3, filesStatus.length);
+    assertEquals("dir-ns", filesStatus[0].getPath().getName());
+    assertEquals("dirns0", filesStatus[1].getPath().getName());
+    assertEquals("dirns1", filesStatus[2].getPath().getName());
   }
 
   @Test
@@ -184,14 +184,11 @@ public class TestDisableNameservices {
         t < TimeUnit.SECONDS.toMillis(1));
     // We should not report anything from ns0
     FileSystem routerFs = routerContext.getFileSystem();
-    FileStatus[] filesStatus = null;
-    try {
-      routerFs.listStatus(new Path("/"));
-      fail("The listStatus call should fail.");
-    } catch (IOException ioe) {
-      GenericTestUtils.assertExceptionContains(
-          "No remote locations available", ioe);
-    }
+
+    FileStatus[] filesStatus = routerFs.listStatus(new Path("/"));
+    assertEquals(2, filesStatus.length);
+    assertEquals("dirns0", filesStatus[0].getPath().getName());
+    assertEquals("dirns1", filesStatus[1].getPath().getName());
 
     filesStatus = routerFs.listStatus(new Path("/dirns1"));
     assertEquals(1, filesStatus.length);
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 b66ff5e..572b33d 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
@@ -269,11 +269,11 @@ public class TestRouterMountTable {
       assertTrue(addMountTable(addEntry));
       addEntry = MountTable.newInstance("/testA/testB",
           Collections.singletonMap("ns0", "/testA/testB"));
+      addEntry.setOwnerName("userB");
+      addEntry.setGroupName("groupB");
       assertTrue(addMountTable(addEntry));
       addEntry = MountTable.newInstance("/testB",
           Collections.singletonMap("ns0", "/test1/testB"));
-      addEntry.setOwnerName("userB");
-      addEntry.setGroupName("groupB");
       assertTrue(addMountTable(addEntry));
 
       assertTrue(nnFs0.mkdirs(new Path("/test1")));
@@ -311,6 +311,44 @@ public class TestRouterMountTable {
     assertEquals("groupB", currentGroup);
   }
 
+  @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")));
+  }
+
+  @Test
+  public void testListWhenDisableDefaultMountTable() throws IOException {
+    mountTable.setDefaultNSEnable(false);
+    /**
+     * /base/dir1 -> ns0:/base/dir1
+     * /base/dir2 -> ns0:/base/dir2
+     */
+    assertTrue(addMountTable(createEntry("/base/dir1", "ns0", "/base/dir1",
+        "group2", "owner2", (short) 0750)));
+    assertTrue(addMountTable(createEntry("/base/dir2", "ns0", "/base/dir2",
+        "group3", "owner3", (short) 0755)));
+
+    FileStatus[] list = routerFs.listStatus(new Path("/base"));
+    assertEquals(2, list.length);
+    for (FileStatus status : list) {
+      if (status.getPath().toUri().getPath().equals("/base/dir1")) {
+        assertEquals("group2", status.getGroup());
+        assertEquals("owner2", status.getOwner());
+        assertEquals((short) 0750, status.getPermission().toShort());
+      } else if (status.getPath().toUri().getPath().equals("/base/dir2")) {
+        assertEquals("group3", status.getGroup());
+        assertEquals("owner3", status.getOwner());
+        assertEquals((short) 0755, status.getPermission().toShort());
+      } else {
+        fail("list result should be either /base/dir1 or /base/dir2.");
+      }
+    }
+  }
+
   /**
    * Verify permission for a mount point when the actual destination is not
    * present. It returns the permissions of the mount point.
@@ -330,6 +368,16 @@ public class TestRouterMountTable {
     assertEquals((short) 0775, list[0].getPermission().toShort());
   }
 
+  private MountTable createEntry(String mountPath, String ns, String remotePath,
+      String group, String owner, short permission) throws IOException {
+    MountTable entry = MountTable
+        .newInstance(mountPath, Collections.singletonMap(ns, remotePath));
+    entry.setGroupName(group);
+    entry.setOwnerName(owner);
+    entry.setMode(FsPermission.createImmutable(permission));
+    return entry;
+  }
+
   /**
    * Verify permission for a mount point when the actual destination present. It
    * returns the permissions of the actual destination pointed by the mount


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