YARN-2247. Made RM web services authenticate users via kerberos and delegation token. Contributed by Varun Vasudev.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1613821 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Zhijie Shen 2014-07-27 17:55:06 +00:00
parent 549bcc2c02
commit d6532d3a77
11 changed files with 1095 additions and 1 deletions

View File

@ -147,6 +147,9 @@ Release 2.5.0 - UNRELEASED
YARN-2233. Implemented ResourceManager web-services to create, renew and
cancel delegation tokens. (Varun Vasudev via vinodkv)
YARN-2247. Made RM web services authenticate users via kerberos and delegation
token. (Varun Vasudev via zjshen)
IMPROVEMENTS
YARN-1479. Invalid NaN values in Hadoop REST API JSON response (Chen He via

View File

@ -263,6 +263,17 @@ public class YarnConfiguration extends Configuration {
public static final String RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY =
RM_PREFIX + "webapp.spnego-keytab-file";
/**
* Flag to enable override of the default kerberos authentication filter with
* the RM authentication filter to allow authentication using delegation
* tokens(fallback to kerberos if the tokens are missing). Only applicable
* when the http authentication type is kerberos.
*/
public static final String RM_WEBAPP_DELEGATION_TOKEN_AUTH_FILTER = RM_PREFIX
+ "webapp.delegation-token-auth-filter.enabled";
public static final boolean DEFAULT_RM_WEBAPP_DELEGATION_TOKEN_AUTH_FILTER =
true;
/** How long to wait until a container is considered dead.*/
public static final String RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS =
RM_PREFIX + "rm.container-allocation.expiry-interval-ms";

View File

@ -194,6 +194,15 @@
<value>/etc/krb5.keytab</value>
</property>
<property>
<description>Flag to enable override of the default kerberos authentication
filter with the RM authentication filter to allow authentication using
delegation tokens(fallback to kerberos if the tokens are missing). Only
applicable when the http authentication type is kerberos.</description>
<name>yarn.resourcemanager.webapp.delegation-token-auth-filter.enabled</name>
<value>true</value>
</property>
<property>
<description>How long to wait until a node manager is considered dead.</description>
<name>yarn.nm.liveness-monitor.expiry-interval-ms</name>

View File

@ -0,0 +1,63 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.security.http;
import java.util.Properties;
import javax.servlet.FilterConfig;
import javax.servlet.ServletException;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
@Private
@Unstable
public class RMAuthenticationFilter extends AuthenticationFilter {
public static final String AUTH_HANDLER_PROPERTY =
"yarn.resourcemanager.authentication-handler";
public RMAuthenticationFilter() {
}
@Override
protected Properties getConfiguration(String configPrefix,
FilterConfig filterConfig) throws ServletException {
// In yarn-site.xml, we can simply set type to "kerberos". However, we need
// to replace the name here to use the customized Kerberos + DT service
// instead of the standard Kerberos handler.
Properties properties = super.getConfiguration(configPrefix, filterConfig);
String yarnAuthHandler = properties.getProperty(AUTH_HANDLER_PROPERTY);
if (yarnAuthHandler == null || yarnAuthHandler.isEmpty()) {
// if http auth type is simple, the default authentication filter
// will handle it, else throw an exception
if (!properties.getProperty(AUTH_TYPE).equals("simple")) {
throw new ServletException("Authentication handler class is empty");
}
}
if (properties.getProperty(AUTH_TYPE).equalsIgnoreCase("kerberos")) {
properties.setProperty(AUTH_TYPE, yarnAuthHandler);
}
return properties;
}
}

View File

@ -0,0 +1,121 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.security.http;
import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStreamReader;
import java.io.Reader;
import java.util.HashMap;
import java.util.Map;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.http.FilterContainer;
import org.apache.hadoop.http.FilterInitializer;
import org.apache.hadoop.http.HttpServer2;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
@Unstable
public class RMAuthenticationFilterInitializer extends FilterInitializer {
String configPrefix;
String signatureSecretFileProperty;
String kerberosPrincipalProperty;
String cookiePath;
public RMAuthenticationFilterInitializer() {
this.configPrefix = "hadoop.http.authentication.";
this.signatureSecretFileProperty =
AuthenticationFilter.SIGNATURE_SECRET + ".file";
this.kerberosPrincipalProperty = KerberosAuthenticationHandler.PRINCIPAL;
this.cookiePath = "/";
}
protected Map<String, String> createFilterConfig(Configuration conf) {
Map<String, String> filterConfig = new HashMap<String, String>();
// setting the cookie path to root '/' so it is used for all resources.
filterConfig.put(AuthenticationFilter.COOKIE_PATH, cookiePath);
for (Map.Entry<String, String> entry : conf) {
String name = entry.getKey();
if (name.startsWith(configPrefix)) {
String value = conf.get(name);
name = name.substring(configPrefix.length());
filterConfig.put(name, value);
}
}
String signatureSecretFile = filterConfig.get(signatureSecretFileProperty);
if (signatureSecretFile != null) {
Reader reader = null;
try {
StringBuilder secret = new StringBuilder();
reader =
new InputStreamReader(new FileInputStream(signatureSecretFile),
"UTF-8");
int c = reader.read();
while (c > -1) {
secret.append((char) c);
c = reader.read();
}
filterConfig.put(AuthenticationFilter.SIGNATURE_SECRET,
secret.toString());
} catch (IOException ex) {
// if running in non-secure mode, this filter only gets added
// because the user has not setup his own filter so just generate
// a random secret. in secure mode, the user needs to setup security
if (UserGroupInformation.isSecurityEnabled()) {
throw new RuntimeException(
"Could not read HTTP signature secret file: " + signatureSecretFile);
}
} finally {
IOUtils.closeQuietly(reader);
}
}
// Resolve _HOST into bind address
String bindAddress = conf.get(HttpServer2.BIND_ADDRESS);
String principal = filterConfig.get(kerberosPrincipalProperty);
if (principal != null) {
try {
principal = SecurityUtil.getServerPrincipal(principal, bindAddress);
} catch (IOException ex) {
throw new RuntimeException(
"Could not resolve Kerberos principal name: " + ex.toString(), ex);
}
filterConfig.put(KerberosAuthenticationHandler.PRINCIPAL, principal);
}
return filterConfig;
}
@Override
public void initFilter(FilterContainer container, Configuration conf) {
Map<String, String> filterConfig = createFilterConfig(conf);
container.addFilter("YARNAuthenticationFilter",
RMAuthenticationFilter.class.getName(), filterConfig);
}
}

View File

@ -32,11 +32,13 @@ import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ha.HAServiceProtocol;
import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState;
import org.apache.hadoop.http.lib.StaticUserWebFilter;
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.metrics2.source.JvmMetrics;
import org.apache.hadoop.security.Groups;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
import org.apache.hadoop.security.authorize.ProxyUsers;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.service.CompositeService;
@ -88,8 +90,11 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEv
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEventType;
import org.apache.hadoop.yarn.server.resourcemanager.security.DelegationTokenRenewer;
import org.apache.hadoop.yarn.server.resourcemanager.security.QueueACLsManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMAuthenticationHandler;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.RMWebApp;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilter;
import org.apache.hadoop.yarn.server.security.http.RMAuthenticationFilterInitializer;
import org.apache.hadoop.yarn.server.webproxy.AppReportFetcher;
import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils;
import org.apache.hadoop.yarn.server.webproxy.WebAppProxy;
@ -789,6 +794,62 @@ public class ResourceManager extends CompositeService implements Recoverable {
}
protected void startWepApp() {
// Use the customized yarn filter instead of the standard kerberos filter to
// allow users to authenticate using delegation tokens
// 3 conditions need to be satisfied -
// 1. security is enabled
// 2. http auth type is set to kerberos
// 3. "yarn.resourcemanager.webapp.use-yarn-filter" override is set to true
Configuration conf = getConfig();
boolean useYarnAuthenticationFilter =
conf.getBoolean(
YarnConfiguration.RM_WEBAPP_DELEGATION_TOKEN_AUTH_FILTER,
YarnConfiguration.DEFAULT_RM_WEBAPP_DELEGATION_TOKEN_AUTH_FILTER);
String authPrefix = "hadoop.http.authentication.";
String authTypeKey = authPrefix + "type";
String initializers = conf.get("hadoop.http.filter.initializers");
if (UserGroupInformation.isSecurityEnabled()
&& useYarnAuthenticationFilter
&& conf.get(authTypeKey, "").equalsIgnoreCase(
KerberosAuthenticationHandler.TYPE)) {
LOG.info("Using RM authentication filter(kerberos/delegation-token)"
+ " for RM webapp authentication");
RMAuthenticationHandler
.setSecretManager(getClientRMService().rmDTSecretManager);
String yarnAuthKey =
authPrefix + RMAuthenticationFilter.AUTH_HANDLER_PROPERTY;
conf.setStrings(yarnAuthKey, RMAuthenticationHandler.class.getName());
initializers =
initializers == null || initializers.isEmpty() ? "" : ","
+ initializers;
if (!initializers.contains(RMAuthenticationFilterInitializer.class
.getName())) {
conf.set("hadoop.http.filter.initializers",
RMAuthenticationFilterInitializer.class.getName() + initializers);
}
}
// if security is not enabled and the default filter initializer has been
// set, set the initializer to include the
// RMAuthenticationFilterInitializer which in turn will set up the simple
// auth filter.
if (!UserGroupInformation.isSecurityEnabled()) {
if (initializers == null || initializers.isEmpty()) {
conf.set("hadoop.http.filter.initializers",
RMAuthenticationFilterInitializer.class.getName());
conf.set(authTypeKey, "simple");
} else if (initializers.equals(StaticUserWebFilter.class.getName())) {
conf.set("hadoop.http.filter.initializers",
RMAuthenticationFilterInitializer.class.getName() + ","
+ initializers);
conf.set(authTypeKey, "simple");
}
}
Builder<ApplicationMasterService> builder =
WebApps
.$for("cluster", ApplicationMasterService.class, masterService,

View File

@ -0,0 +1,157 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.security;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException;
import javax.servlet.http.HttpServletRequest;
import javax.servlet.http.HttpServletResponse;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.client.AuthenticationException;
import org.apache.hadoop.security.authentication.server.AuthenticationToken;
import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
public class RMAuthenticationHandler extends KerberosAuthenticationHandler {
public static final String TYPE = "kerberos-dt";
public static final String HEADER = "Hadoop-YARN-Auth-Delegation-Token";
static RMDelegationTokenSecretManager secretManager;
static boolean secretManagerInitialized = false;
public RMAuthenticationHandler() {
super();
}
/**
* Returns authentication type of the handler.
*
* @return <code>kerberos-dt</code>
*/
@Override
public String getType() {
return TYPE;
}
@Override
public boolean managementOperation(AuthenticationToken token,
HttpServletRequest request, HttpServletResponse response) {
return true;
}
/**
* Authenticates a request looking for the <code>delegation</code> header and
* verifying it is a valid token. If the header is missing, it delegates the
* authentication to the {@link KerberosAuthenticationHandler} unless it is
* disabled.
*
* @param request
* the HTTP client request.
* @param response
* the HTTP client response.
*
* @return the authentication token for the authenticated request.
* @throws IOException
* thrown if an IO error occurred.
* @throws AuthenticationException
* thrown if the authentication failed.
*/
@Override
public AuthenticationToken authenticate(HttpServletRequest request,
HttpServletResponse response) throws IOException, AuthenticationException {
AuthenticationToken token;
String delegationParam = this.getEncodedDelegationTokenFromRequest(request);
if (delegationParam != null) {
Token<RMDelegationTokenIdentifier> dt =
new Token<RMDelegationTokenIdentifier>();
;
dt.decodeFromUrlString(delegationParam);
UserGroupInformation ugi = this.verifyToken(dt);
if (ugi == null) {
throw new AuthenticationException("Invalid token");
}
final String shortName = ugi.getShortUserName();
token = new AuthenticationToken(shortName, ugi.getUserName(), getType());
} else {
token = super.authenticate(request, response);
if (token != null) {
// create a token with auth type set correctly
token =
new AuthenticationToken(token.getUserName(), token.getName(),
super.getType());
}
}
return token;
}
/**
* Verifies a delegation token.
*
* @param token
* delegation token to verify.
* @return the UGI for the token; null if the verification fails
* @throws IOException
* thrown if the token could not be verified.
*/
protected UserGroupInformation verifyToken(
Token<RMDelegationTokenIdentifier> token) throws IOException {
if (secretManagerInitialized == false) {
throw new IllegalStateException("Secret manager not initialized");
}
ByteArrayInputStream buf = new ByteArrayInputStream(token.getIdentifier());
DataInputStream dis = new DataInputStream(buf);
RMDelegationTokenIdentifier id = secretManager.createIdentifier();
try {
id.readFields(dis);
secretManager.verifyToken(id, token.getPassword());
} catch (Throwable t) {
return null;
} finally {
dis.close();
}
return id.getUser();
}
/**
* Extract encoded delegation token from request
*
* @param req
* HTTPServletRequest object
*
* @return String containing the encoded token; null if encoded token not
* found
*
*/
protected String getEncodedDelegationTokenFromRequest(HttpServletRequest req) {
String header = req.getHeader(HEADER);
return header;
}
public static void setSecretManager(RMDelegationTokenSecretManager manager) {
secretManager = manager;
secretManagerInitialized = true;
}
}

View File

@ -55,6 +55,7 @@ import org.apache.commons.codec.binary.Base64;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.io.DataOutputBuffer;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.Credentials;
@ -680,6 +681,11 @@ public class RMWebServices {
throw new AuthorizationException(msg);
}
if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
String msg = "The default static user cannot carry out this operation.";
return Response.status(Status.FORBIDDEN).entity(msg).build();
}
String userName = callerUGI.getUserName();
RMApp app = null;
try {
@ -800,6 +806,13 @@ public class RMWebServices {
return callerUGI;
}
private boolean isStaticUser(UserGroupInformation callerUGI) {
String staticUser =
conf.get(CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER,
CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER);
return staticUser.equals(callerUGI.getUserName());
}
/**
* Generates a new ApplicationId which is then sent to the client
*
@ -822,6 +835,10 @@ public class RMWebServices {
throw new AuthorizationException("Unable to obtain user name, "
+ "user not authenticated");
}
if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
String msg = "The default static user cannot carry out this operation.";
return Response.status(Status.FORBIDDEN).entity(msg).build();
}
NewApplication appId = createNewApplication();
return Response.status(Status.OK).entity(appId).build();
@ -859,6 +876,11 @@ public class RMWebServices {
+ "user not authenticated");
}
if (UserGroupInformation.isSecurityEnabled() && isStaticUser(callerUGI)) {
String msg = "The default static user cannot carry out this operation.";
return Response.status(Status.FORBIDDEN).entity(msg).build();
}
ApplicationSubmissionContext appContext =
createAppSubmissionContext(newApp);
final SubmitApplicationRequest req =
@ -975,7 +997,7 @@ public class RMWebServices {
*
* @param newApp
* the information provided by the user
* @return
* @return created context
* @throws BadRequestException
* @throws IOException
*/

View File

@ -0,0 +1,354 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.webapp;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.BufferedReader;
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.io.InputStreamReader;
import java.io.OutputStream;
import java.io.StringWriter;
import java.net.HttpURLConnection;
import java.net.URL;
import java.util.concurrent.Callable;
import javax.ws.rs.core.MediaType;
import javax.xml.bind.JAXBContext;
import javax.xml.bind.Marshaller;
import org.apache.commons.io.IOUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.KerberosTestUtils;
import org.apache.hadoop.security.authentication.server.AuthenticationFilter;
import org.apache.hadoop.security.authentication.server.KerberosAuthenticationHandler;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.codehaus.jettison.json.JSONObject;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import com.sun.jersey.api.client.ClientResponse.Status;
public class TestRMWebServicesDelegationTokenAuthentication {
private static final File testRootDir = new File("target",
TestRMWebServicesDelegationTokenAuthentication.class.getName() + "-root");
private static File httpSpnegoKeytabFile = new File(
KerberosTestUtils.getKeytabFile());
private static String httpSpnegoPrincipal = KerberosTestUtils
.getServerPrincipal();
private static boolean miniKDCStarted = false;
private static MiniKdc testMiniKDC;
private static MockRM rm;
// use published header name
final static String DelegationTokenHeader =
"Hadoop-YARN-Auth-Delegation-Token";
@BeforeClass
public static void setUp() {
try {
testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
setupKDC();
setupAndStartRM();
} catch (Exception e) {
assertTrue("Couldn't create MiniKDC", false);
}
}
@AfterClass
public static void tearDown() {
if (testMiniKDC != null) {
testMiniKDC.stop();
}
if (rm != null) {
rm.stop();
}
}
public TestRMWebServicesDelegationTokenAuthentication() throws Exception {
super();
}
private static void setupAndStartRM() throws Exception {
Configuration rmconf = new Configuration();
rmconf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
rmconf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
ResourceScheduler.class);
rmconf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
String httpPrefix = "hadoop.http.authentication.";
rmconf.setStrings(httpPrefix + "type", "kerberos");
rmconf.set(httpPrefix + KerberosAuthenticationHandler.PRINCIPAL,
httpSpnegoPrincipal);
rmconf.set(httpPrefix + KerberosAuthenticationHandler.KEYTAB,
httpSpnegoKeytabFile.getAbsolutePath());
// use any file for signature secret
rmconf.set(httpPrefix + AuthenticationFilter.SIGNATURE_SECRET + ".file",
httpSpnegoKeytabFile.getAbsolutePath());
rmconf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos");
rmconf.setBoolean(YarnConfiguration.RM_WEBAPP_DELEGATION_TOKEN_AUTH_FILTER,
true);
rmconf.set(YarnConfiguration.RM_WEBAPP_SPNEGO_USER_NAME_KEY,
httpSpnegoPrincipal);
rmconf.set(YarnConfiguration.RM_KEYTAB,
httpSpnegoKeytabFile.getAbsolutePath());
rmconf.set(YarnConfiguration.RM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,
httpSpnegoKeytabFile.getAbsolutePath());
rmconf.set(YarnConfiguration.NM_WEBAPP_SPNEGO_USER_NAME_KEY,
httpSpnegoPrincipal);
rmconf.set(YarnConfiguration.NM_WEBAPP_SPNEGO_KEYTAB_FILE_KEY,
httpSpnegoKeytabFile.getAbsolutePath());
rmconf.setBoolean("mockrm.webapp.enabled", true);
UserGroupInformation.setConfiguration(rmconf);
rm = new MockRM(rmconf);
rm.start();
}
private static void setupKDC() throws Exception {
if (miniKDCStarted == false) {
testMiniKDC.start();
getKdc().createPrincipal(httpSpnegoKeytabFile, "HTTP/localhost",
"client", UserGroupInformation.getLoginUser().getShortUserName());
miniKDCStarted = true;
}
}
private static MiniKdc getKdc() {
return testMiniKDC;
}
// Test that you can authenticate with only delegation tokens
// 1. Get a delegation token using Kerberos auth(this ends up
// testing the fallback authenticator)
// 2. Submit an app without kerberos or delegation-token
// - we should get an UNAUTHORIZED response
// 3. Submit same app with delegation-token
// - we should get OK response
// - confirm owner of the app is the user whose
// delegation-token we used
@Test
public void testDelegationTokenAuth() throws Exception {
final String token = getDelegationToken("test");
ApplicationSubmissionContextInfo app =
new ApplicationSubmissionContextInfo();
String appid = "application_123_0";
app.setApplicationId(appid);
String requestBody = getMarshalledAppInfo(app);
URL url = new URL("http://localhost:8088/ws/v1/cluster/apps");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
setupConn(conn, "POST", "application/xml", requestBody);
// this should fail with unauthorized because only
// auth is kerberos or delegation token
try {
conn.getInputStream();
fail("we should not be here");
} catch (IOException e) {
assertEquals(Status.UNAUTHORIZED.getStatusCode(), conn.getResponseCode());
}
conn = (HttpURLConnection) url.openConnection();
conn.setRequestProperty(DelegationTokenHeader, token);
setupConn(conn, "POST", MediaType.APPLICATION_XML, requestBody);
// this should not fail
conn.getInputStream();
boolean appExists =
rm.getRMContext().getRMApps()
.containsKey(ConverterUtils.toApplicationId(appid));
assertTrue(appExists);
RMApp actualApp =
rm.getRMContext().getRMApps()
.get(ConverterUtils.toApplicationId(appid));
String owner = actualApp.getUser();
assertEquals("client", owner);
return;
}
// Test to make sure that cancelled delegation tokens
// are rejected
@Test
public void testCancelledDelegationToken() throws Exception {
String token = getDelegationToken("client");
cancelDelegationToken(token);
ApplicationSubmissionContextInfo app =
new ApplicationSubmissionContextInfo();
String appid = "application_123_0";
app.setApplicationId(appid);
String requestBody = getMarshalledAppInfo(app);
URL url = new URL("http://localhost:8088/ws/v1/cluster/apps");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
conn.setRequestProperty(DelegationTokenHeader, token);
setupConn(conn, "POST", MediaType.APPLICATION_XML, requestBody);
// this should fail with unauthorized because only
// auth is kerberos or delegation token
try {
conn.getInputStream();
fail("Authentication should fail with expired delegation tokens");
} catch (IOException e) {
assertEquals(Status.FORBIDDEN.getStatusCode(), conn.getResponseCode());
}
return;
}
// Test to make sure that we can't do delegation token
// functions using just delegation token auth
@Test
public void testDelegationTokenOps() throws Exception {
String token = getDelegationToken("client");
String createRequest = "{\"renewer\":\"test\"}";
String renewRequest = "{\"token\": \"" + token + "\"}";
// first test create and renew
String[] requests = { createRequest, renewRequest };
for (String requestBody : requests) {
URL url = new URL("http://localhost:8088/ws/v1/cluster/delegation-token");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
conn.setRequestProperty(DelegationTokenHeader, token);
setupConn(conn, "POST", MediaType.APPLICATION_JSON, requestBody);
try {
conn.getInputStream();
fail("Creation/Renewing delegation tokens should not be "
+ "allowed with token auth");
} catch (IOException e) {
assertEquals(Status.FORBIDDEN.getStatusCode(), conn.getResponseCode());
}
}
// test cancel
URL url = new URL("http://localhost:8088/ws/v1/cluster/delegation-token");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
conn.setRequestProperty(DelegationTokenHeader, token);
conn.setRequestProperty(RMWebServices.DELEGATION_TOKEN_HEADER, token);
setupConn(conn, "DELETE", null, null);
try {
conn.getInputStream();
fail("Cancelling delegation tokens should not be allowed with token auth");
} catch (IOException e) {
assertEquals(Status.FORBIDDEN.getStatusCode(), conn.getResponseCode());
}
return;
}
private String getDelegationToken(final String renewer) throws Exception {
String token = KerberosTestUtils.doAsClient(new Callable<String>() {
@Override
public String call() throws Exception {
String ret = null;
String body = "{\"renewer\":\"" + renewer + "\"}";
URL url =
new URL("http://localhost:8088/ws/v1/cluster/delegation-token");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
setupConn(conn, "POST", MediaType.APPLICATION_JSON, body);
InputStream response = conn.getInputStream();
assertEquals(Status.OK.getStatusCode(), conn.getResponseCode());
BufferedReader reader = null;
try {
reader = new BufferedReader(new InputStreamReader(response, "UTF8"));
for (String line; (line = reader.readLine()) != null;) {
JSONObject obj = new JSONObject(line);
if (obj.has("token")) {
reader.close();
response.close();
ret = obj.getString("token");
break;
}
}
} finally {
IOUtils.closeQuietly(reader);
IOUtils.closeQuietly(response);
}
return ret;
}
});
return token;
}
private void cancelDelegationToken(final String tokenString) throws Exception {
KerberosTestUtils.doAsClient(new Callable<Void>() {
@Override
public Void call() throws Exception {
URL url =
new URL("http://localhost:8088/ws/v1/cluster/delegation-token");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
conn.setRequestProperty(RMWebServices.DELEGATION_TOKEN_HEADER,
tokenString);
setupConn(conn, "DELETE", null, null);
InputStream response = conn.getInputStream();
assertEquals(Status.OK.getStatusCode(), conn.getResponseCode());
response.close();
return null;
}
});
return;
}
static String getMarshalledAppInfo(ApplicationSubmissionContextInfo appInfo)
throws Exception {
StringWriter writer = new StringWriter();
JAXBContext context =
JAXBContext.newInstance(ApplicationSubmissionContextInfo.class);
Marshaller m = context.createMarshaller();
m.marshal(appInfo, writer);
return writer.toString();
}
static void setupConn(HttpURLConnection conn, String method,
String contentType, String body) throws Exception {
conn.setRequestMethod(method);
conn.setDoOutput(true);
conn.setRequestProperty("Accept-Charset", "UTF8");
if (contentType != null && !contentType.isEmpty()) {
conn.setRequestProperty("Content-Type", contentType + ";charset=UTF8");
if (body != null && !body.isEmpty()) {
OutputStream stream = conn.getOutputStream();
stream.write(body.getBytes("UTF8"));
stream.close();
}
}
}
}

View File

@ -0,0 +1,272 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.yarn.server.resourcemanager.webapp;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
import java.io.File;
import java.io.IOException;
import java.net.HttpURLConnection;
import java.net.URL;
import java.util.Arrays;
import java.util.Collection;
import javax.ws.rs.core.MediaType;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeys;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.minikdc.MiniKdc;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authentication.KerberosTestUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.webapp.dao.ApplicationSubmissionContextInfo;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
import com.sun.jersey.api.client.ClientResponse.Status;
/* Just a simple test class to ensure that the RM handles the static web user
* correctly for secure and un-secure modes
*
*/
@RunWith(Parameterized.class)
public class TestRMWebappAuthentication {
private static MockRM rm;
private static Configuration simpleConf;
private static Configuration kerberosConf;
private static final File testRootDir = new File("target",
TestRMWebServicesDelegationTokenAuthentication.class.getName() + "-root");
private static File httpSpnegoKeytabFile = new File(
KerberosTestUtils.getKeytabFile());
private static boolean miniKDCStarted = false;
private static MiniKdc testMiniKDC;
static {
simpleConf = new Configuration();
simpleConf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
simpleConf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
ResourceScheduler.class);
simpleConf.setBoolean("mockrm.webapp.enabled", true);
kerberosConf = new Configuration();
kerberosConf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS,
YarnConfiguration.DEFAULT_RM_AM_MAX_ATTEMPTS);
kerberosConf.setClass(YarnConfiguration.RM_SCHEDULER, FifoScheduler.class,
ResourceScheduler.class);
kerberosConf.setBoolean(YarnConfiguration.YARN_ACL_ENABLE, true);
kerberosConf.set(
CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
kerberosConf.set(YarnConfiguration.RM_KEYTAB,
httpSpnegoKeytabFile.getAbsolutePath());
kerberosConf.setBoolean("mockrm.webapp.enabled", true);
}
@Parameters
public static Collection params() {
return Arrays.asList(new Object[][] { { 1, simpleConf },
{ 2, kerberosConf } });
}
public TestRMWebappAuthentication(int run, Configuration conf) {
super();
setupAndStartRM(conf);
}
@BeforeClass
public static void setUp() {
try {
testMiniKDC = new MiniKdc(MiniKdc.createConf(), testRootDir);
setupKDC();
} catch (Exception e) {
assertTrue("Couldn't create MiniKDC", false);
}
}
@AfterClass
public static void tearDown() {
if (testMiniKDC != null) {
testMiniKDC.stop();
}
}
private static void setupKDC() throws Exception {
if (!miniKDCStarted) {
testMiniKDC.start();
getKdc().createPrincipal(httpSpnegoKeytabFile, "HTTP/localhost",
"client", UserGroupInformation.getLoginUser().getShortUserName());
miniKDCStarted = true;
}
}
private static MiniKdc getKdc() {
return testMiniKDC;
}
private static void setupAndStartRM(Configuration conf) {
UserGroupInformation.setConfiguration(conf);
rm = new MockRM(conf);
}
// ensure that in a non-secure cluster users can access
// the web pages as earlier and submit apps as anonymous
// user or by identifying themselves
@Test
public void testSimpleAuth() throws Exception {
rm.start();
// ensure users can access web pages
// this should work for secure and non-secure clusters
URL url = new URL("http://localhost:8088/cluster");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
try {
conn.getInputStream();
assertEquals(Status.OK.getStatusCode(), conn.getResponseCode());
} catch (Exception e) {
fail("Fetching url failed");
}
if (UserGroupInformation.isSecurityEnabled()) {
testAnonymousKerberosUser();
} else {
testAnonymousSimpleUser();
}
rm.stop();
}
private void testAnonymousKerberosUser() throws Exception {
ApplicationSubmissionContextInfo app =
new ApplicationSubmissionContextInfo();
String appid = "application_123_0";
app.setApplicationId(appid);
String requestBody =
TestRMWebServicesDelegationTokenAuthentication
.getMarshalledAppInfo(app);
URL url =
new URL("http://localhost:8088/ws/v1/cluster/apps/new-application");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
TestRMWebServicesDelegationTokenAuthentication.setupConn(conn, "POST",
"application/xml", requestBody);
try {
conn.getInputStream();
fail("Anonymous users should not be allowed to get new application ids in secure mode.");
} catch (IOException ie) {
assertEquals(Status.FORBIDDEN.getStatusCode(), conn.getResponseCode());
}
url = new URL("http://localhost:8088/ws/v1/cluster/apps");
conn = (HttpURLConnection) url.openConnection();
TestRMWebServicesDelegationTokenAuthentication.setupConn(conn, "POST",
"application/xml", requestBody);
try {
conn.getInputStream();
fail("Anonymous users should not be allowed to submit apps in secure mode.");
} catch (IOException ie) {
assertEquals(Status.FORBIDDEN.getStatusCode(), conn.getResponseCode());
}
requestBody = "{ \"state\": \"KILLED\"}";
url =
new URL(
"http://localhost:8088/ws/v1/cluster/apps/application_123_0/state");
conn = (HttpURLConnection) url.openConnection();
TestRMWebServicesDelegationTokenAuthentication.setupConn(conn, "PUT",
"application/json", requestBody);
try {
conn.getInputStream();
fail("Anonymous users should not be allowed to kill apps in secure mode.");
} catch (IOException ie) {
assertEquals(Status.FORBIDDEN.getStatusCode(), conn.getResponseCode());
}
}
private void testAnonymousSimpleUser() throws Exception {
ApplicationSubmissionContextInfo app =
new ApplicationSubmissionContextInfo();
String appid = "application_123_0";
app.setApplicationId(appid);
String requestBody =
TestRMWebServicesDelegationTokenAuthentication
.getMarshalledAppInfo(app);
URL url = new URL("http://localhost:8088/ws/v1/cluster/apps");
HttpURLConnection conn = (HttpURLConnection) url.openConnection();
TestRMWebServicesDelegationTokenAuthentication.setupConn(conn, "POST",
"application/xml", requestBody);
conn.getInputStream();
assertEquals(Status.ACCEPTED.getStatusCode(), conn.getResponseCode());
boolean appExists =
rm.getRMContext().getRMApps()
.containsKey(ConverterUtils.toApplicationId(appid));
assertTrue(appExists);
RMApp actualApp =
rm.getRMContext().getRMApps()
.get(ConverterUtils.toApplicationId(appid));
String owner = actualApp.getUser();
assertEquals(
rm.getConfig().get(CommonConfigurationKeys.HADOOP_HTTP_STATIC_USER,
CommonConfigurationKeys.DEFAULT_HADOOP_HTTP_STATIC_USER), owner);
appid = "application_123_1";
app.setApplicationId(appid);
requestBody =
TestRMWebServicesDelegationTokenAuthentication
.getMarshalledAppInfo(app);
url = new URL("http://localhost:8088/ws/v1/cluster/apps?user.name=client");
conn = (HttpURLConnection) url.openConnection();
TestRMWebServicesDelegationTokenAuthentication.setupConn(conn, "POST",
MediaType.APPLICATION_XML, requestBody);
conn.getInputStream();
appExists =
rm.getRMContext().getRMApps()
.containsKey(ConverterUtils.toApplicationId(appid));
assertTrue(appExists);
actualApp =
rm.getRMContext().getRMApps()
.get(ConverterUtils.toApplicationId(appid));
owner = actualApp.getUser();
assertEquals("client", owner);
}
}

View File

@ -2912,3 +2912,24 @@ Accept: application/xml
+---+
No response body.
** Authentication using delegation tokens
This feature is in the alpha mode and may change in the future.
You can use delegation tokens to authenticate yourself when using YARN RM webservices. However, this requires setting the right configurations. The conditions for this are:
* Hadoop is setup in secure mode with the authentication type set to kerberos.
* Hadoop HTTP authentication is setup with the authentication type set to kerberos
Once setup, delegation tokens can be fetched using the web services listed above and used as shown in an example below:
+---+
PUT http://<rm http address:port>/ws/v1/cluster/apps/application_1399397633663_0003/state
Hadoop-YARN-Auth-Delegation-Token: MgASY2xpZW50QEVYQU1QTEUuQ09NDHRlc3QtcmVuZXdlcgCKAUbjqcHHigFHB7ZFxwQCFKWD3znCkDSy6SQIjRCLDydxbxvgE1JNX0RFTEVHQVRJT05fVE9LRU4A
Content-Type: application/json; charset=UTF8
{
"state":"KILLED"
}
+---+