Skip to content

Commit

Permalink
HDFS-13996. Make HttpFS' ACLs RegEx configurable. Contributed by Siya…
Browse files Browse the repository at this point in the history
…o Meng.

Signed-off-by: Wei-Chiu Chuang <weichiu@apache.org>
  • Loading branch information
smengcl authored and jojochuang committed Nov 2, 2018
1 parent d98b881 commit 8fe85af
Show file tree
Hide file tree
Showing 5 changed files with 156 additions and 7 deletions.
Expand Up @@ -22,6 +22,8 @@
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;
Expand All @@ -37,8 +39,6 @@
import java.util.Map;
import java.util.regex.Pattern;

import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEBHDFS_ACL_PERMISSION_PATTERN_DEFAULT;

/**
* HttpFS ParametersProvider.
*/
Expand Down Expand Up @@ -430,7 +430,11 @@ public static class AclPermissionParam extends StringParam {
*/
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)));
}
}

Expand Down
Expand Up @@ -38,6 +38,8 @@
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;
Expand Down Expand Up @@ -117,6 +119,14 @@ private void createHttpFSServer() throws Exception {
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);
Expand Down Expand Up @@ -1130,6 +1140,7 @@ private void operation(Operation op) throws Exception {
testContentSummary();
break;
case FILEACLS:
testFileAclsCustomizedUserAndGroupNames();
testFileAcls();
break;
case DIRACLS:
Expand Down Expand Up @@ -1581,4 +1592,53 @@ private void testGetSnapshottableDirListing() throws Exception {
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);
}
}
Expand Up @@ -19,9 +19,11 @@

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;
Expand Down Expand Up @@ -199,8 +201,24 @@ private Configuration createHttpFSConf(boolean addDelegationTokenAuthHandler,
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);
Expand All @@ -213,6 +231,7 @@ private void createHttpFSServer(boolean addDelegationTokenAuthHandler,
if (addDelegationTokenAuthHandler) {
HttpFSServerWebApp.get().setAuthority(TestJettyHelper.getAuthority());
}
return server;
}

private String getSignedTokenString()
Expand Down Expand Up @@ -891,6 +910,48 @@ public void testDirAcls() throws Exception {
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
Expand Down
Expand Up @@ -32,13 +32,16 @@
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;
import org.junit.runners.model.Statement;

public class TestHdfsHelper extends TestDirHelper {

private HdfsStatement statement;

@Override
@Test
public void dummy() {
Expand All @@ -54,23 +57,32 @@ public void dummy() {
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);
Expand Down Expand Up @@ -162,6 +174,14 @@ private static synchronized MiniDFSCluster startMiniHdfs(Configuration conf) thr
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");
Expand Down
Expand Up @@ -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 =
Expand Down

0 comments on commit 8fe85af

Please sign in to comment.