Selaa lähdekoodia

HDDS-29. Fix TestStorageContainerManager#testRpcPermission. Contributed by Mukul Kumar Singh.

Xiaoyu Yao 7 vuotta sitten
vanhempi
commit
fc5d49c202

+ 13 - 2
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/SCMClientProtocolServer.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ozone.protocolPB
     .StorageContainerLocationProtocolServerSideTranslatorPB;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -137,17 +138,26 @@ public class SCMClientProtocolServer implements
     getClientRpcServer().join();
   }
 
+  @VisibleForTesting
+  public String getRpcRemoteUsername() {
+    UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
+    return user == null ? null : user.getUserName();
+  }
+
   @Override
   public ContainerInfo allocateContainer(HddsProtos.ReplicationType
       replicationType, HddsProtos.ReplicationFactor factor,
       String owner) throws IOException {
-    getScm().checkAdminAccess();
+    String remoteUser = getRpcRemoteUsername();
+    getScm().checkAdminAccess(remoteUser);
     return scm.getScmContainerManager()
         .allocateContainer(replicationType, factor, owner);
   }
 
   @Override
   public ContainerInfo getContainer(long containerID) throws IOException {
+    String remoteUser = getRpcRemoteUsername();
+    getScm().checkAdminAccess(remoteUser);
     return scm.getScmContainerManager()
         .getContainer(containerID);
   }
@@ -161,7 +171,8 @@ public class SCMClientProtocolServer implements
 
   @Override
   public void deleteContainer(long containerID) throws IOException {
-    getScm().checkAdminAccess();
+    String remoteUser = getRpcRemoteUsername();
+    getScm().checkAdminAccess(remoteUser);
     scm.getScmContainerManager().deleteContainer(containerID);
 
   }

+ 1 - 8
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManager.java

@@ -620,14 +620,7 @@ public class StorageContainerManager extends ServiceRuntimeInfoImpl
     return scmBlockManager;
   }
 
-  @VisibleForTesting
-  public String getPpcRemoteUsername() {
-    UserGroupInformation user = ProtobufRpcEngine.Server.getRemoteUser();
-    return user == null ? null : user.getUserName();
-  }
-
-  public void checkAdminAccess() throws IOException {
-    String remoteUser = getPpcRemoteUsername();
+  public void checkAdminAccess(String remoteUser) throws IOException {
     if (remoteUser != null) {
       if (!scmAdminUsernames.contains(remoteUser)) {
         throw new IOException(

+ 14 - 13
hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/TestStorageContainerManager.java

@@ -23,6 +23,7 @@ import java.io.IOException;
 import org.apache.commons.lang3.RandomUtils;
 import org.apache.hadoop.hdds.conf.OzoneConfiguration;
 import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerInfo;
+import org.apache.hadoop.hdds.scm.server.SCMClientProtocolServer;
 import org.apache.hadoop.hdds.scm.server.SCMStorage;
 import org.apache.hadoop.hdds.scm.node.NodeManager;
 import org.apache.hadoop.ozone.container.ContainerTestHelper;
@@ -107,19 +108,19 @@ public class TestStorageContainerManager {
     MiniOzoneCluster cluster = MiniOzoneCluster.newBuilder(ozoneConf).build();
     cluster.waitForClusterToBeReady();
     try {
-      String fakeUser = fakeRemoteUsername;
-      StorageContainerManager mockScm = Mockito.spy(
-          cluster.getStorageContainerManager());
-      Mockito.when(mockScm.getPpcRemoteUsername())
-          .thenReturn(fakeUser);
+
+      SCMClientProtocolServer mockClientServer = Mockito.spy(
+          cluster.getStorageContainerManager().getClientProtocolServer());
+      Mockito.when(mockClientServer.getRpcRemoteUsername())
+          .thenReturn(fakeRemoteUsername);
 
       try {
-        mockScm.getClientProtocolServer().deleteContainer(
+        mockClientServer.deleteContainer(
             ContainerTestHelper.getTestContainerID());
         fail("Operation should fail, expecting an IOException here.");
       } catch (Exception e) {
         if (expectPermissionDenied) {
-          verifyPermissionDeniedException(e, fakeUser);
+          verifyPermissionDeniedException(e, fakeRemoteUsername);
         } else {
           // If passes permission check, it should fail with
           // container not exist exception.
@@ -129,7 +130,7 @@ public class TestStorageContainerManager {
       }
 
       try {
-        ContainerInfo container2 = mockScm.getClientProtocolServer()
+        ContainerInfo container2 = mockClientServer
             .allocateContainer(xceiverClientManager.getType(),
             HddsProtos.ReplicationFactor.ONE,  "OZONE");
         if (expectPermissionDenied) {
@@ -138,11 +139,11 @@ public class TestStorageContainerManager {
           Assert.assertEquals(1, container2.getPipeline().getMachines().size());
         }
       } catch (Exception e) {
-        verifyPermissionDeniedException(e, fakeUser);
+        verifyPermissionDeniedException(e, fakeRemoteUsername);
       }
 
       try {
-        ContainerInfo container3 = mockScm.getClientProtocolServer()
+        ContainerInfo container3 = mockClientServer
             .allocateContainer(xceiverClientManager.getType(),
             HddsProtos.ReplicationFactor.ONE, "OZONE");
         if (expectPermissionDenied) {
@@ -151,16 +152,16 @@ public class TestStorageContainerManager {
           Assert.assertEquals(1, container3.getPipeline().getMachines().size());
         }
       } catch (Exception e) {
-        verifyPermissionDeniedException(e, fakeUser);
+        verifyPermissionDeniedException(e, fakeRemoteUsername);
       }
 
       try {
-        mockScm.getClientProtocolServer().getContainer(
+        mockClientServer.getContainer(
             ContainerTestHelper.getTestContainerID());
         fail("Operation should fail, expecting an IOException here.");
       } catch (Exception e) {
         if (expectPermissionDenied) {
-          verifyPermissionDeniedException(e, fakeUser);
+          verifyPermissionDeniedException(e, fakeRemoteUsername);
         } else {
           // If passes permission check, it should fail with
           // key not exist exception.