瀏覽代碼

HDFS-12062. removeErasureCodingPolicy needs super user permission. Contributed by Wei-Chiu Chuang.

Wei-Chiu Chuang 7 年之前
父節點
當前提交
369f731264

+ 22 - 4
hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java

@@ -2774,25 +2774,43 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   public AddECPolicyResponse[] addErasureCodingPolicies(
       ErasureCodingPolicy[] policies) throws IOException {
     checkOpen();
-    return namenode.addErasureCodingPolicies(policies);
+    try (TraceScope ignored = tracer.newScope("addErasureCodingPolicies")) {
+      return namenode.addErasureCodingPolicies(policies);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
   }
 
   public void removeErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkOpen();
-    namenode.removeErasureCodingPolicy(ecPolicyName);
+    try (TraceScope ignored = tracer.newScope("removeErasureCodingPolicy")) {
+      namenode.removeErasureCodingPolicy(ecPolicyName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class);
+    }
   }
 
   public void enableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkOpen();
-    namenode.enableErasureCodingPolicy(ecPolicyName);
+    try (TraceScope ignored = tracer.newScope("enableErasureCodingPolicy")) {
+      namenode.enableErasureCodingPolicy(ecPolicyName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
+    }
   }
 
   public void disableErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkOpen();
-    namenode.disableErasureCodingPolicy(ecPolicyName);
+    try (TraceScope ignored = tracer.newScope("disableErasureCodingPolicy")) {
+      namenode.disableErasureCodingPolicy(ecPolicyName);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class);
+    }
   }
 
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {

+ 1 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java

@@ -2304,6 +2304,7 @@ public class NameNodeRpcServer implements NamenodeProtocols {
   public void removeErasureCodingPolicy(String ecPolicyName)
       throws IOException {
     checkNNStartup();
+    namesystem.checkSuperuserPrivilege();
     namesystem.removeErasureCodingPolicy(ecPolicyName);
   }
 

+ 50 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDistributedFileSystem.java

@@ -93,6 +93,7 @@ import org.apache.hadoop.net.DNSToSwitchMapping;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.net.ScriptBasedMapping;
 import org.apache.hadoop.net.StaticMapping;
+import org.apache.hadoop.security.AccessControlException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.test.GenericTestUtils;
 import org.apache.hadoop.util.DataChecksum;
@@ -1561,6 +1562,27 @@ public class TestDistributedFileSystem {
       fs.removeErasureCodingPolicy(policyName);
       assertEquals(policyName, ErasureCodingPolicyManager.getInstance().
           getRemovedPolicies().get(0).getName());
+
+      // remove erasure coding policy as a user without privilege
+      UserGroupInformation fakeUGI = UserGroupInformation.createUserForTesting(
+          "ProbablyNotARealUserName", new String[] {"ShangriLa"});
+      final MiniDFSCluster finalCluster = cluster;
+      fakeUGI.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          DistributedFileSystem fs = finalCluster.getFileSystem();
+          try {
+            fs.removeErasureCodingPolicy(policyName);
+            fail();
+          } catch (AccessControlException ace) {
+            GenericTestUtils.assertExceptionContains("Access denied for user " +
+                "ProbablyNotARealUserName. Superuser privilege is required",
+                ace);
+          }
+          return null;
+        }
+      });
+
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -1609,6 +1631,34 @@ public class TestDistributedFileSystem {
         GenericTestUtils.assertExceptionContains("does not exists", e);
         // pass
       }
+
+      // disable and enable erasure coding policy as a user without privilege
+      UserGroupInformation fakeUGI = UserGroupInformation.createUserForTesting(
+          "ProbablyNotARealUserName", new String[] {"ShangriLa"});
+      final MiniDFSCluster finalCluster = cluster;
+      fakeUGI.doAs(new PrivilegedExceptionAction<Object>() {
+        @Override
+        public Object run() throws Exception {
+          DistributedFileSystem fs = finalCluster.getFileSystem();
+          try {
+            fs.disableErasureCodingPolicy(policyName);
+            fail();
+          } catch (AccessControlException ace) {
+            GenericTestUtils.assertExceptionContains("Access denied for user " +
+                    "ProbablyNotARealUserName. Superuser privilege is required",
+                ace);
+          }
+          try {
+            fs.enableErasureCodingPolicy(policyName);
+            fail();
+          } catch (AccessControlException ace) {
+            GenericTestUtils.assertExceptionContains("Access denied for user " +
+                    "ProbablyNotARealUserName. Superuser privilege is required",
+                ace);
+          }
+          return null;
+        }
+      });
     } finally {
       if (cluster != null) {
         cluster.shutdown();

+ 20 - 0
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java

@@ -693,5 +693,25 @@ public class TestErasureCodingPolicies {
     assertTrue(responses[0].isSucceed());
     assertEquals(SystemErasureCodingPolicies.getPolicies().size() + 1,
         ErasureCodingPolicyManager.getInstance().getPolicies().length);
+
+    // add erasure coding policy as a user without privilege
+    UserGroupInformation fakeUGI = UserGroupInformation.createUserForTesting(
+        "ProbablyNotARealUserName", new String[] {"ShangriLa"});
+    final ErasureCodingPolicy ecPolicy = newPolicy;
+    fakeUGI.doAs(new PrivilegedExceptionAction<Object>() {
+      @Override
+      public Object run() throws Exception {
+        DistributedFileSystem fs = cluster.getFileSystem();
+        try {
+          fs.addErasureCodingPolicies(new ErasureCodingPolicy[]{ecPolicy});
+          fail();
+        } catch (AccessControlException ace) {
+          GenericTestUtils.assertExceptionContains("Access denied for user " +
+                  "ProbablyNotARealUserName. Superuser privilege is required",
+              ace);
+        }
+        return null;
+      }
+    });
   }
 }