瀏覽代碼

HDFS-13996. Make HttpFS' ACLs RegEx configurable. Contributed by Siyao Meng.

Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
Siyao Meng 6 年之前
父節點
當前提交
8fe85af63b

+ 7 - 3
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/fs/http/server/HttpFSParametersProvider.java

@@ -22,6 +22,8 @@ import org.apache.hadoop.fs.XAttrCodec;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem;
 import org.apache.hadoop.fs.http.client.HttpFSFileSystem.Operation;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
+import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.lib.wsrs.BooleanParam;
 import org.apache.hadoop.lib.wsrs.EnumParam;
 import org.apache.hadoop.lib.wsrs.EnumSetParam;
@@ -37,8 +39,6 @@ import java.util.HashMap;
 import java.util.Map;
 import java.util.regex.Pattern;
 
-import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
-
 /**
  * HttpFS ParametersProvider.
  */
@@ -430,7 +430,11 @@ public class HttpFSParametersProvider extends ParametersProvider {
      */
     public AclPermissionParam() {
       super(NAME, HttpFSFileSystem.ACLSPEC_DEFAULT,
-              Pattern.compile(DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT));
+        Pattern.compile(HttpFSServerWebApp.get()
+          .get(FileSystemAccess.class)
+          .getFileSystemConfiguration()
+          .get(HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_KEY,
+            HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT)));
     }
   }
 

+ 60 - 0
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/client/BaseTestHttpFSWith.java

@@ -38,6 +38,8 @@ import org.apache.hadoop.hdfs.AppendTestUtil;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshotException;
@@ -117,6 +119,14 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
     conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, fsDefaultName);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
     conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
+    // For BaseTestHttpFSWith#testFileAclsCustomizedUserAndGroupNames
+    conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
+        "^[A-Za-z0-9_][A-Za-z0-9._-]*[$]?$");
+    conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_KEY,
+        "^(default:)?(user|group|mask|other):" +
+            "[[0-9A-Za-z_][@A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?" +
+            "(user|group|mask|other):[[0-9A-Za-z_][@A-Za-z0-9._-]]*:" +
+            "([rwx-]{3})?)*$");
     File hdfsSite = new File(new File(homeDir, "conf"), "hdfs-site.xml");
     OutputStream os = new FileOutputStream(hdfsSite);
     conf.writeXml(os);
@@ -1130,6 +1140,7 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       testContentSummary();
       break;
     case FILEACLS:
+      testFileAclsCustomizedUserAndGroupNames();
       testFileAcls();
       break;
     case DIRACLS:
@@ -1581,4 +1592,53 @@ public abstract class BaseTestHttpFSWith extends HFSTestCase {
       verifyGetSnapshottableDirListing(fs, dfs);
     }
   }
+
+  private void testFileAclsCustomizedUserAndGroupNames() throws Exception {
+    if (isLocalFS()) {
+      return;
+    }
+
+    // Get appropriate conf from the cluster
+    MiniDFSCluster miniDFSCluster = ((TestHdfsHelper) hdfsTestHelper)
+        .getMiniDFSCluster();
+    Configuration conf = miniDFSCluster.getConfiguration(0);
+    // If we call getHttpFSFileSystem() without conf from the mini cluster,
+    // WebHDFS will be initialized with the default ACL string, causing the
+    // setAcl() later to fail. This is only an issue in the unit test.
+    FileSystem httpfs = getHttpFSFileSystem(conf);
+    if (!(httpfs instanceof WebHdfsFileSystem)
+        && !(httpfs instanceof HttpFSFileSystem)) {
+      Assert.fail(httpfs.getClass().getSimpleName() +
+          " doesn't support custom user and group name pattern. "
+          + "Only WebHdfsFileSystem and HttpFSFileSystem support it.");
+    }
+    final String aclUser = "user:123:rwx";
+    final String aclGroup = "group:foo@bar:r--";
+    final String aclSet = "user::rwx," + aclUser + ",group::r--," +
+        aclGroup + ",other::r--";
+    final String dir = "/aclFileTestCustom";
+    // Create test file
+    FileSystem proxyFs = FileSystem.get(conf);
+    proxyFs.mkdirs(new Path(dir));
+    Path path = new Path(dir, "/testACL");
+    OutputStream os = proxyFs.create(path);
+    os.write(1);
+    os.close();
+    // Set ACL
+    httpfs.setAcl(path, AclEntry.parseAclSpec(aclSet, true));
+    // Verify getAclStatus responses are the same
+    AclStatus proxyAclStat = proxyFs.getAclStatus(path);
+    AclStatus httpfsAclStat = httpfs.getAclStatus(path);
+    assertSameAcls(httpfsAclStat, proxyAclStat);
+    assertSameAcls(httpfs, proxyFs, path);
+    // Verify that custom user and group are set.
+    List<String> strEntries = new ArrayList<>();
+    for (AclEntry aclEntry : httpfsAclStat.getEntries()) {
+      strEntries.add(aclEntry.toStringStable());
+    }
+    Assert.assertTrue(strEntries.contains(aclUser));
+    Assert.assertTrue(strEntries.contains(aclGroup));
+    // Clean up
+    proxyFs.delete(new Path(dir), true);
+  }
 }

+ 63 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/fs/http/server/TestHttpFSServer.java

@@ -19,9 +19,11 @@ package org.apache.hadoop.fs.http.server;
 
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
 import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus;
 import org.apache.hadoop.hdfs.web.JsonUtil;
+import org.apache.hadoop.lib.service.FileSystemAccess;
 import org.apache.hadoop.security.authentication.util.SignerSecretProvider;
 import org.apache.hadoop.security.authentication.util.StringSignerSecretProviderCreator;
 import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier;
@@ -199,8 +201,24 @@ public class TestHttpFSServer extends HFSTestCase {
     return conf;
   }
 
-  private void createHttpFSServer(boolean addDelegationTokenAuthHandler,
-                                  boolean sslEnabled)
+  /**
+   * Write configuration to a site file under Hadoop configuration dir.
+   */
+  private void writeConf(Configuration conf, String sitename)
+      throws Exception {
+    File homeDir = TestDirHelper.getTestDir();
+    // HDFS configuration
+    File hadoopConfDir = new File(new File(homeDir, "conf"), "hadoop-conf");
+    Assert.assertTrue(hadoopConfDir.exists());
+
+    File siteFile = new File(hadoopConfDir, sitename);
+    OutputStream os = new FileOutputStream(siteFile);
+    conf.writeXml(os);
+    os.close();
+  }
+
+  private Server createHttpFSServer(boolean addDelegationTokenAuthHandler,
+                                    boolean sslEnabled)
       throws Exception {
     Configuration conf = createHttpFSConf(addDelegationTokenAuthHandler,
                                           sslEnabled);
@@ -213,6 +231,7 @@ public class TestHttpFSServer extends HFSTestCase {
     if (addDelegationTokenAuthHandler) {
       HttpFSServerWebApp.get().setAuthority(TestJettyHelper.getAuthority());
     }
+    return server;
   }
 
   private String getSignedTokenString()
@@ -891,6 +910,48 @@ public class TestHttpFSServer extends HFSTestCase {
     Assert.assertTrue(aclEntries.size() == 0);
   }
 
+  @Test
+  @TestDir
+  @TestJetty
+  @TestHdfs
+  public void testCustomizedUserAndGroupNames() throws Exception {
+    // Start server with default configuration
+    Server server = createHttpFSServer(false, false);
+    final Configuration conf = HttpFSServerWebApp.get()
+        .get(FileSystemAccess.class).getFileSystemConfiguration();
+    // Change pattern config
+    conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
+        "^[A-Za-z0-9_][A-Za-z0-9._-]*[$]?$");
+    conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_KEY,
+        "^(default:)?(user|group|mask|other):" +
+            "[[0-9A-Za-z_][@A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?" +
+            "(user|group|mask|other):[[0-9A-Za-z_][@A-Za-z0-9._-]]*:" +
+            "([rwx-]{3})?)*$");
+    // Save configuration to site file
+    writeConf(conf, "hdfs-site.xml");
+    // Restart the HttpFS server to apply new config
+    server.stop();
+    server.start();
+
+    final String aclUser = "user:123:rw-";
+    final String aclGroup = "group:foo@bar:r--";
+    final String aclSpec = "aclspec=user::rwx," + aclUser + ",group::rwx," +
+        aclGroup + ",other::---";
+    final String dir = "/aclFileTestCustom";
+    final String path = dir + "/test";
+    // Create test dir
+    FileSystem fs = FileSystem.get(conf);
+    fs.mkdirs(new Path(dir));
+    createWithHttp(path, null);
+    // Set ACL
+    putCmd(path, "SETACL", aclSpec);
+    // Verify ACL
+    String statusJson = getStatus(path, "GETACLSTATUS");
+    List<String> aclEntries = getAclEntries(statusJson);
+    Assert.assertTrue(aclEntries.contains(aclUser));
+    Assert.assertTrue(aclEntries.contains(aclGroup));
+  }
+
   @Test
   @TestDir
   @TestJetty

+ 22 - 2
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/test/java/org/apache/hadoop/test/TestHdfsHelper.java

@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.StripedFileTestUtil;
+import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys;
 import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy;
 import org.junit.Test;
 import org.junit.runners.model.FrameworkMethod;
@@ -39,6 +40,8 @@ import org.junit.runners.model.Statement;
 
 public class TestHdfsHelper extends TestDirHelper {
 
+  private HdfsStatement statement;
+
   @Override
   @Test
   public void dummy() {
@@ -54,23 +57,32 @@ public class TestHdfsHelper extends TestDirHelper {
   public Statement apply(Statement statement, FrameworkMethod frameworkMethod, Object o) {
     TestHdfs testHdfsAnnotation = frameworkMethod.getAnnotation(TestHdfs.class);
     if (testHdfsAnnotation != null) {
-      statement = new HdfsStatement(statement, frameworkMethod.getName());
+      this.statement = new HdfsStatement(statement, frameworkMethod.getName());
+      statement = this.statement;
     }
     return super.apply(statement, frameworkMethod, o);
   }
 
+  public MiniDFSCluster getMiniDFSCluster() {
+    return statement.getMiniDFSCluster();
+  }
+
   private static class HdfsStatement extends Statement {
     private Statement statement;
     private String testName;
+    private MiniDFSCluster miniHdfs = null;
 
     public HdfsStatement(Statement statement, String testName) {
       this.statement = statement;
       this.testName = testName;
     }
 
+    public MiniDFSCluster getMiniDFSCluster() {
+      return miniHdfs;
+    }
+
     @Override
     public void evaluate() throws Throwable {
-      MiniDFSCluster miniHdfs = null;
       Configuration conf = HadoopUsersConfTestHelper.getBaseConf();
       if (Boolean.parseBoolean(System.getProperty(HADOOP_MINI_HDFS, "true"))) {
         miniHdfs = startMiniHdfs(conf);
@@ -162,6 +174,14 @@ public class TestHdfsHelper extends TestDirHelper {
       conf.set("hadoop.security.authentication", "simple");
       conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_ACLS_ENABLED_KEY, true);
       conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_XATTRS_ENABLED_KEY, true);
+      // For BaseTestHttpFSWith#testFileAclsCustomizedUserAndGroupNames
+      conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_USER_PATTERN_KEY,
+          "^[A-Za-z0-9_][A-Za-z0-9._-]*[$]?$");
+      conf.set(HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_KEY,
+          "^(default:)?(user|group|mask|other):" +
+              "[[0-9A-Za-z_][@A-Za-z0-9._-]]*:([rwx-]{3})?(,(default:)?" +
+              "(user|group|mask|other):[[0-9A-Za-z_][@A-Za-z0-9._-]]*:" +
+              "([rwx-]{3})?)*$");
       FileSystemTestHelper helper = new FileSystemTestHelper();
       Path targetFile = new Path(new File(helper.getTestRootDir())
           .getAbsolutePath(), "test.jks");

+ 4 - 0
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java

@@ -62,6 +62,10 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       .DFS_CHECKSUM_TYPE_KEY;
   public static final String  DFS_CHECKSUM_TYPE_DEFAULT =
       HdfsClientConfigKeys.DFS_CHECKSUM_TYPE_DEFAULT;
+  @Deprecated
+  public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_KEY =
+      HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_KEY;
+  @Deprecated
   public static final String DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT =
       HdfsClientConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;
   public static final String  DFS_WEBHDFS_NETTY_LOW_WATERMARK =