HDFS-14609. RBF: Security should use common AuthenticationFilter. Contributed by Chen Zhang.
This commit is contained in:
parent
1ada99b0bd
commit
a79f286c6f
|
@ -24,8 +24,6 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_HTTPS_ADDRESS_KE
|
|||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KERBEROS_PRINCIPAL_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_KEYTAB_FILE_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_KEYSTORE_RESOURCE_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
|
||||
import static org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.DFS_DATA_TRANSFER_PROTECTION_KEY;
|
||||
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY;
|
||||
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KERBEROS_PRINCIPAL_KEY;
|
||||
|
@ -65,6 +63,7 @@ public final class SecurityConfUtil {
|
|||
// State string for mini dfs
|
||||
private static final String SPNEGO_USER_NAME = "HTTP";
|
||||
private static final String ROUTER_USER_NAME = "router";
|
||||
private static final String PREFIX = "hadoop.http.authentication.";
|
||||
|
||||
private static String spnegoPrincipal;
|
||||
private static String routerPrincipal;
|
||||
|
@ -73,6 +72,10 @@ public final class SecurityConfUtil {
|
|||
// Utility Class
|
||||
}
|
||||
|
||||
public static String getRouterUserName() {
|
||||
return ROUTER_USER_NAME;
|
||||
}
|
||||
|
||||
public static Configuration initSecurity() throws Exception {
|
||||
// delete old test dir
|
||||
File baseDir = GenericTestUtils.getTestDir(
|
||||
|
@ -114,8 +117,9 @@ public final class SecurityConfUtil {
|
|||
conf.set(DFS_NAMENODE_KEYTAB_FILE_KEY, keytab);
|
||||
conf.set(DFS_DATANODE_KERBEROS_PRINCIPAL_KEY, routerPrincipal);
|
||||
conf.set(DFS_DATANODE_KEYTAB_FILE_KEY, keytab);
|
||||
conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY, spnegoPrincipal);
|
||||
conf.set(DFS_WEB_AUTHENTICATION_KERBEROS_KEYTAB_KEY, keytab);
|
||||
conf.set(PREFIX + "type", "kerberos");
|
||||
conf.set(PREFIX + "kerberos.principal", spnegoPrincipal);
|
||||
conf.set(PREFIX + "kerberos.keytab", keytab);
|
||||
|
||||
conf.set(DFS_NAMENODE_HTTPS_ADDRESS_KEY, "localhost:0");
|
||||
conf.set(DFS_DATANODE_HTTPS_ADDRESS_KEY, "localhost:0");
|
||||
|
@ -138,7 +142,8 @@ public final class SecurityConfUtil {
|
|||
// Setup principals and keytabs for router
|
||||
conf.set(DFS_ROUTER_KEYTAB_FILE_KEY, keytab);
|
||||
conf.set(DFS_ROUTER_KERBEROS_PRINCIPAL_KEY, routerPrincipal);
|
||||
conf.set(DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY, "*");
|
||||
conf.set(DFS_ROUTER_KERBEROS_INTERNAL_SPNEGO_PRINCIPAL_KEY,
|
||||
spnegoPrincipal);
|
||||
|
||||
// Setup basic state store
|
||||
conf.setClass(RBFConfigKeys.FEDERATION_STORE_DRIVER_CLASS,
|
||||
|
|
|
@ -27,7 +27,6 @@ import org.junit.rules.ExpectedException;
|
|||
import java.io.IOException;
|
||||
|
||||
import static org.apache.hadoop.fs.contract.router.SecurityConfUtil.initSecurity;
|
||||
import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY;
|
||||
import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_ROUTER_KEYTAB_FILE_KEY;
|
||||
|
||||
|
||||
|
@ -36,13 +35,22 @@ import static org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys.DFS_
|
|||
*/
|
||||
public class TestRouterWithSecureStartup {
|
||||
|
||||
private static final String HTTP_KERBEROS_PRINCIPAL_CONF_KEY =
|
||||
"hadoop.http.authentication.kerberos.principal";
|
||||
|
||||
@Rule
|
||||
public ExpectedException exceptionRule = ExpectedException.none();
|
||||
|
||||
/*
|
||||
* hadoop.http.authentication.kerberos.principal has default value, so if we
|
||||
* don't config the spnego principal, cluster will still start normally
|
||||
*/
|
||||
@Test
|
||||
public void testStartupWithoutSpnegoPrincipal() throws Exception {
|
||||
testCluster(DFS_WEB_AUTHENTICATION_KERBEROS_PRINCIPAL_KEY,
|
||||
"Unable to initialize WebAppContext");
|
||||
Configuration conf = initSecurity();
|
||||
conf.unset(HTTP_KERBEROS_PRINCIPAL_CONF_KEY);
|
||||
RouterWebHDFSContract.createCluster(conf);
|
||||
assertNotNull(RouterWebHDFSContract.getCluster());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -14,6 +14,7 @@
|
|||
|
||||
package org.apache.hadoop.hdfs.server.federation.security;
|
||||
|
||||
import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.HADOOP_HTTP_AUTHENTICATION_TYPE;
|
||||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.assertNotNull;
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
@ -21,9 +22,15 @@ import static org.junit.Assert.assertTrue;
|
|||
import java.io.ByteArrayInputStream;
|
||||
import java.io.DataInputStream;
|
||||
import java.io.IOException;
|
||||
import java.net.HttpURLConnection;
|
||||
import java.net.InetSocketAddress;
|
||||
import java.net.URI;
|
||||
import java.net.URL;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Arrays;
|
||||
import java.util.Enumeration;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Properties;
|
||||
|
||||
import javax.servlet.FilterConfig;
|
||||
|
@ -33,15 +40,23 @@ import org.apache.hadoop.conf.Configuration;
|
|||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.SWebHdfs;
|
||||
import org.apache.hadoop.fs.contract.router.SecurityConfUtil;
|
||||
import org.apache.hadoop.hdfs.DFSConfigKeys;
|
||||
import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
|
||||
import org.apache.hadoop.hdfs.server.federation.RouterConfigBuilder;
|
||||
import org.apache.hadoop.hdfs.server.federation.router.RBFConfigKeys;
|
||||
import org.apache.hadoop.hdfs.server.federation.router.Router;
|
||||
import org.apache.hadoop.hdfs.web.WebHdfsFileSystem;
|
||||
import org.apache.hadoop.hdfs.web.WebHdfsTestUtil;
|
||||
import org.apache.hadoop.hdfs.web.resources.GetOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.HttpOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.Param;
|
||||
import org.apache.hadoop.hdfs.web.resources.PutOpParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.RenewerParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.TokenArgumentParam;
|
||||
import org.apache.hadoop.hdfs.web.resources.UserParam;
|
||||
import org.apache.hadoop.http.FilterContainer;
|
||||
import org.apache.hadoop.security.AuthenticationFilterInitializer;
|
||||
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
|
||||
import org.apache.hadoop.security.authentication.server.PseudoAuthenticationHandler;
|
||||
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.test.LambdaTestUtils;
|
||||
import org.junit.After;
|
||||
|
@ -54,9 +69,26 @@ import org.junit.Test;
|
|||
*/
|
||||
public class TestRouterHttpDelegationToken {
|
||||
|
||||
public static final String FILTER_INITIALIZER_PROPERTY =
|
||||
"hadoop.http.filter.initializers";
|
||||
private Router router;
|
||||
private WebHdfsFileSystem fs;
|
||||
|
||||
/**
|
||||
* The initializer of custom filter.
|
||||
*/
|
||||
public static final class NoAuthFilterInitializer
|
||||
extends AuthenticationFilterInitializer {
|
||||
static final String PREFIX = "hadoop.http.authentication.";
|
||||
|
||||
@Override
|
||||
public void initFilter(FilterContainer container, Configuration conf) {
|
||||
Map<String, String> filterConfig = getFilterConfigMap(conf, PREFIX);
|
||||
container.addFilter("authentication", NoAuthFilter.class.getName(),
|
||||
filterConfig);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Custom filter to be able to test auth methods and let the other ones go.
|
||||
*/
|
||||
|
@ -85,8 +117,9 @@ public class TestRouterHttpDelegationToken {
|
|||
conf.set(RBFConfigKeys.DFS_ROUTER_HTTP_ADDRESS_KEY, "0.0.0.0:0");
|
||||
conf.set(RBFConfigKeys.DFS_ROUTER_HTTPS_ADDRESS_KEY, "0.0.0.0:0");
|
||||
conf.set(RBFConfigKeys.DFS_ROUTER_RPC_ADDRESS_KEY, "0.0.0.0:0");
|
||||
conf.set(DFSConfigKeys.DFS_WEBHDFS_AUTHENTICATION_FILTER_KEY,
|
||||
NoAuthFilter.class.getName());
|
||||
conf.set(FILTER_INITIALIZER_PROPERTY,
|
||||
NoAuthFilterInitializer.class.getName());
|
||||
conf.set(HADOOP_HTTP_AUTHENTICATION_TYPE, "simple");
|
||||
|
||||
// Start routers with an RPC and HTTP service only
|
||||
Configuration routerConf = new RouterConfigBuilder()
|
||||
|
@ -116,7 +149,7 @@ public class TestRouterHttpDelegationToken {
|
|||
@Test
|
||||
public void testGetDelegationToken() throws Exception {
|
||||
final String renewer = "renewer0";
|
||||
Token<?> token = fs.getDelegationToken(renewer);
|
||||
Token<?> token = getDelegationToken(fs, renewer);
|
||||
assertNotNull(token);
|
||||
|
||||
DelegationTokenIdentifier tokenId =
|
||||
|
@ -126,30 +159,70 @@ public class TestRouterHttpDelegationToken {
|
|||
assertEquals("", tokenId.getRealUser().toString());
|
||||
assertEquals("SWEBHDFS delegation", token.getKind().toString());
|
||||
assertNotNull(token.getPassword());
|
||||
|
||||
InetSocketAddress webAddress = router.getHttpServerAddress();
|
||||
assertEquals(webAddress.getHostName() + ":" + webAddress.getPort(),
|
||||
token.getService().toString());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testRenewDelegationToken() throws Exception {
|
||||
Token<?> token = fs.getDelegationToken("router");
|
||||
Token<?> token = getDelegationToken(fs, "router");
|
||||
DelegationTokenIdentifier tokenId =
|
||||
getTokenIdentifier(token.getIdentifier());
|
||||
|
||||
long t = fs.renewDelegationToken(token);
|
||||
long t = renewDelegationToken(fs, token);
|
||||
assertTrue(t + " should not be larger than " + tokenId.getMaxDate(),
|
||||
t <= tokenId.getMaxDate());
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCancelDelegationToken() throws Exception {
|
||||
Token<?> token = fs.getDelegationToken("router");
|
||||
fs.cancelDelegationToken(token);
|
||||
LambdaTestUtils.intercept(InvalidToken.class,
|
||||
"Renewal request for unknown token",
|
||||
() -> fs.renewDelegationToken(token));
|
||||
Token<?> token = getDelegationToken(fs, "router");
|
||||
cancelDelegationToken(fs, token);
|
||||
LambdaTestUtils.intercept(IOException.class,
|
||||
"Server returned HTTP response code: 403 ",
|
||||
() -> renewDelegationToken(fs, token));
|
||||
}
|
||||
|
||||
private Token<DelegationTokenIdentifier> getDelegationToken(
|
||||
WebHdfsFileSystem webHdfs, String renewer) throws IOException {
|
||||
Map<?, ?> json = sendHttpRequest(webHdfs, GetOpParam.Op.GETDELEGATIONTOKEN,
|
||||
new RenewerParam(renewer));
|
||||
return WebHdfsTestUtil.convertJsonToDelegationToken(json);
|
||||
}
|
||||
|
||||
private long renewDelegationToken(WebHdfsFileSystem webHdfs, Token<?> token)
|
||||
throws IOException {
|
||||
Map<?, ?> json =
|
||||
sendHttpRequest(webHdfs, PutOpParam.Op.RENEWDELEGATIONTOKEN,
|
||||
new TokenArgumentParam(token.encodeToUrlString()));
|
||||
return ((Number) json.get("long")).longValue();
|
||||
}
|
||||
|
||||
private void cancelDelegationToken(WebHdfsFileSystem webHdfs, Token<?> token)
|
||||
throws IOException {
|
||||
sendHttpRequest(webHdfs, PutOpParam.Op.CANCELDELEGATIONTOKEN,
|
||||
new TokenArgumentParam(token.encodeToUrlString()));
|
||||
}
|
||||
|
||||
private Map<?, ?> sendHttpRequest(WebHdfsFileSystem webHdfs,
|
||||
final HttpOpParam.Op op, final Param<?, ?>... parameters)
|
||||
throws IOException {
|
||||
String user = SecurityConfUtil.getRouterUserName();
|
||||
// process parameters, add user.name
|
||||
List<Param<?, ?>> pList = new ArrayList<>();
|
||||
pList.add(new UserParam(user));
|
||||
pList.addAll(Arrays.asList(parameters));
|
||||
|
||||
// build request url
|
||||
final URL url = WebHdfsTestUtil.toUrl(webHdfs, op, null,
|
||||
pList.toArray(new Param<?, ?>[pList.size()]));
|
||||
|
||||
// open connection and send request
|
||||
HttpURLConnection conn =
|
||||
WebHdfsTestUtil.openConnection(url, webHdfs.getConf());
|
||||
conn.setRequestMethod(op.getType().toString());
|
||||
WebHdfsTestUtil.sendRequest(conn);
|
||||
final Map<?, ?> json = WebHdfsTestUtil.getAndParseResponse(conn);
|
||||
conn.disconnect();
|
||||
return json;
|
||||
}
|
||||
|
||||
private DelegationTokenIdentifier getTokenIdentifier(byte[] id)
|
||||
|
|
Loading…
Reference in New Issue