Prechádzať zdrojové kódy

HDFS-14815. RBF: Update the quota in MountTable when calling setQuota on a MountTable src. Contributed by Jinglun.

Ayush Saxena 5 rokov pred
rodič
commit
42fc8884ab

+ 21 - 3
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/Quota.java

@@ -33,6 +33,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.federation.resolver.RemoteLocation;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
+import org.apache.hadoop.security.AccessControlException;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -68,10 +69,17 @@ public class Quota {
    * @param namespaceQuota Name space quota.
    * @param storagespaceQuota Storage space quota.
    * @param type StorageType that the space quota is intended to be set on.
-   * @throws IOException If the quota system is disabled.
+   * @param checkMountEntry whether to check the path is a mount entry.
+   * @throws AccessControlException If the quota system is disabled or if
+   * checkMountEntry is true and the path is a mount entry.
    */
-  public void setQuota(String path, long namespaceQuota,
-      long storagespaceQuota, StorageType type) throws IOException {
+  public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
+      StorageType type, boolean checkMountEntry) throws IOException {
+    if (checkMountEntry && isMountEntry(path)) {
+      throw new AccessControlException(
+          "Permission denied: " + RouterRpcServer.getRemoteUser()
+              + " is not allowed to change quota of " + path);
+    }
     setQuotaInternal(path, null, namespaceQuota, storagespaceQuota, type);
   }
 
@@ -174,6 +182,16 @@ public class Quota {
     return new QuotaUsage.Builder().quota(nQuota).spaceQuota(sQuota).build();
   }
 
+  /**
+   * Is the path a mount entry.
+   *
+   * @param path the path to be checked.
+   * @return {@code true} if path is a mount entry; {@code false} otherwise.
+   */
+  private boolean isMountEntry(String path) {
+    return router.getQuotaManager().isMountEntry(path);
+  }
+
   /**
    * Get valid quota remote locations used in {@link #getQuotaUsage(String)}.
    * Differentiate the method {@link #getQuotaRemoteLocations(String)}, this

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

@@ -345,7 +345,7 @@ public class RouterAdminServer extends AbstractService
         ((StateStoreCache) this.router.getSubclusterResolver()).loadCache(true);
       }
       Quota routerQuota = this.router.getRpcServer().getQuotaModule();
-      routerQuota.setQuota(path, nsQuota, ssQuota, null);
+      routerQuota.setQuota(path, nsQuota, ssQuota, null, false);
     }
   }
 

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

@@ -1585,7 +1585,7 @@ public class RouterClientProtocol implements ClientProtocol {
   public void setQuota(String path, long namespaceQuota, long storagespaceQuota,
       StorageType type) throws IOException {
     rpcServer.getQuotaModule()
-        .setQuota(path, namespaceQuota, storagespaceQuota, type);
+        .setQuota(path, namespaceQuota, storagespaceQuota, type, true);
   }
 
   @Override

+ 15 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/main/java/org/apache/hadoop/hdfs/server/federation/router/RouterQuotaManager.java

@@ -62,6 +62,21 @@ public class RouterQuotaManager {
     }
   }
 
+  /**
+   * Is the path a mount entry.
+   *
+   * @param path the path.
+   * @return {@code true} if path is a mount entry; {@code false} otherwise.
+   */
+  boolean isMountEntry(String path) {
+    readLock.lock();
+    try {
+      return this.cache.containsKey(path);
+    } finally {
+      readLock.unlock();
+    }
+  }
+
   /**
    * Get the nearest ancestor's quota usage, and meanwhile its quota was set.
    * @param path The path being written.

+ 1 - 1
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestDisableRouterQuota.java

@@ -73,7 +73,7 @@ public class TestDisableRouterQuota {
 
     try {
       Quota quotaModule = router.getRpcServer().getQuotaModule();
-      quotaModule.setQuota("/test", nsQuota, ssQuota, null);
+      quotaModule.setQuota("/test", nsQuota, ssQuota, null, false);
       fail("The setQuota call should fail.");
     } catch (IOException ioe) {
       GenericTestUtils.assertExceptionContains(

+ 3 - 2
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterAdminCLI.java

@@ -123,8 +123,9 @@ public class TestRouterAdminCLI {
     // Mock the quota module since no real namenode is started up.
     Quota quota = Mockito
         .spy(routerContext.getRouter().createRpcServer().getQuotaModule());
-    Mockito.doNothing().when(quota).setQuota(Mockito.anyString(),
-        Mockito.anyLong(), Mockito.anyLong(), Mockito.any());
+    Mockito.doNothing().when(quota)
+        .setQuota(Mockito.anyString(), Mockito.anyLong(), Mockito.anyLong(),
+            Mockito.any(), Mockito.anyBoolean());
     Whitebox.setInternalState(
         routerContext.getRouter().getRpcServer(), "quotaCall", quota);
 

+ 53 - 0
hadoop-hdfs-project/hadoop-hdfs-rbf/src/test/java/org/apache/hadoop/hdfs/server/federation/router/TestRouterQuota.java

@@ -37,6 +37,7 @@ import org.apache.hadoop.fs.CreateFlag;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.QuotaUsage;
+import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSClient;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
@@ -62,7 +63,9 @@ import org.apache.hadoop.hdfs.server.federation.store.protocol.RemoveMountTableE
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryRequest;
 import org.apache.hadoop.hdfs.server.federation.store.protocol.UpdateMountTableEntryResponse;
 import org.apache.hadoop.hdfs.server.federation.store.records.MountTable;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.test.GenericTestUtils;
+import org.apache.hadoop.test.LambdaTestUtils;
 import org.apache.hadoop.util.Time;
 import org.junit.After;
 import org.junit.Before;
@@ -282,6 +285,56 @@ public class TestRouterQuota {
     stream.close();
   }
 
+  @Test
+  public void testSetQuotaToMountTableEntry() throws Exception {
+    long nsQuota = 10;
+    long ssQuota = 10240;
+    long diskQuota = 1024;
+    final FileSystem nnFs1 = nnContext1.getFileSystem();
+    nnFs1.mkdirs(new Path("/testSetQuotaToFederationPath"));
+    nnFs1.mkdirs(new Path("/testSetQuotaToFederationPath/dir0"));
+
+    // Add one mount table:
+    // /setquota --> ns0---testSetQuotaToFederationPath
+    MountTable mountTable = MountTable.newInstance("/setquota",
+        Collections.singletonMap("ns0", "/testSetQuotaToFederationPath"));
+    addMountTable(mountTable);
+
+    RouterQuotaUpdateService updateService = routerContext.getRouter()
+        .getQuotaCacheUpdateService();
+    // ensure mount table is updated to Router.
+    updateService.periodicInvoke();
+
+    final FileSystem routerFs = routerContext.getFileSystem();
+    // setting quota on a mount point should fail.
+    LambdaTestUtils.intercept(AccessControlException.class,
+        "is not allowed to change quota of",
+        "Expect an AccessControlException.",
+        () -> routerFs.setQuota(new Path("/setquota"), nsQuota, ssQuota));
+    // setting storage quota on a mount point should fail.
+    LambdaTestUtils.intercept(AccessControlException.class,
+        "is not allowed to change quota of",
+        "Expect an AccessControlException.", () -> routerFs
+            .setQuotaByStorageType(new Path("/setquota"), StorageType.DISK,
+                diskQuota));
+    QuotaUsage quota =
+        nnFs1.getQuotaUsage(new Path("/testSetQuotaToFederationPath"));
+    // quota should still be unset.
+    assertEquals(-1, quota.getQuota());
+    assertEquals(-1, quota.getSpaceQuota());
+
+    // setting quota on a non-mount point should succeed.
+    routerFs.setQuota(new Path("/setquota/dir0"), nsQuota, ssQuota);
+    // setting storage quota on a non-mount point should succeed.
+    routerFs.setQuotaByStorageType(new Path("/setquota/dir0"), StorageType.DISK,
+        diskQuota);
+    quota = nnFs1.getQuotaUsage(new Path("/testSetQuotaToFederationPath/dir0"));
+    // quota should be set.
+    assertEquals(nsQuota, quota.getQuota());
+    assertEquals(ssQuota, quota.getSpaceQuota());
+    assertEquals(diskQuota, quota.getTypeQuota(StorageType.DISK));
+  }
+
   @Test
   public void testSetQuota() throws Exception {
     long nsQuota = 5;