Skip to content

Commit

Permalink
YARN-2247. Made RM web services authenticate users via kerberos and d…
Browse files Browse the repository at this point in the history
…elegation token. Contributed by Varun Vasudev.

svn merge --ignore-ancestry -c 1613821 ../../trunk/


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2.5@1613823 13f79535-47bb-0310-9956-ffa450edef68
  • Loading branch information
zjshen14 committed Jul 27, 2014
1 parent 7d32a9d commit ceb7cfb
Show file tree
Hide file tree
Showing 11 changed files with 1,095 additions and 1 deletion.
3 changes: 3 additions & 0 deletions hadoop-yarn-project/CHANGES.txt
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,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
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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";
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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>
Expand Down
Original file line number Diff line number Diff line change
@@ -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;
}

}
Original file line number Diff line number Diff line change
@@ -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);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -32,11 +32,13 @@
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;
Expand Down Expand Up @@ -88,8 +90,11 @@
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;
Expand Down Expand Up @@ -789,6 +794,62 @@ public void handle(RMNodeEvent event) {
}

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,
Expand Down
Loading

0 comments on commit ceb7cfb

Please sign in to comment.