YARN-2798. Fixed YarnClient to populate the renewer correctly for Timeline delegation tokens. Contributed by Zhijie Shen.
(cherry picked from commit 71fbb474f5
)
This commit is contained in:
parent
8091ea54d5
commit
52c58115d2
|
@ -812,6 +812,9 @@ Release 2.6.0 - UNRELEASED
|
|||
YARN-2730. DefaultContainerExecutor runs only one localizer at a time
|
||||
(Siqi Li via jlowe)
|
||||
|
||||
YARN-2798. Fixed YarnClient to populate the renewer correctly for Timeline
|
||||
delegation tokens. (Zhijie Shen via vinodkv)
|
||||
|
||||
Release 2.5.1 - 2014-09-05
|
||||
|
||||
INCOMPATIBLE CHANGES
|
||||
|
|
|
@ -36,7 +36,7 @@ import org.apache.hadoop.io.DataOutputBuffer;
|
|||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.ipc.RPC;
|
||||
import org.apache.hadoop.security.Credentials;
|
||||
import org.apache.hadoop.security.HadoopKerberosName;
|
||||
import org.apache.hadoop.security.SecurityUtil;
|
||||
import org.apache.hadoop.security.UserGroupInformation;
|
||||
import org.apache.hadoop.security.token.TokenIdentifier;
|
||||
import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
|
||||
|
@ -51,7 +51,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationsResponse;
|
|||
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterMetricsResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodeLabelsResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesRequest;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetClusterNodesResponse;
|
||||
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerReportRequest;
|
||||
|
@ -124,6 +123,8 @@ public class YarnClientImpl extends YarnClient {
|
|||
protected TimelineClient timelineClient;
|
||||
@VisibleForTesting
|
||||
Text timelineService;
|
||||
@VisibleForTesting
|
||||
String timelineDTRenewer;
|
||||
protected boolean timelineServiceEnabled;
|
||||
|
||||
private static final String ROOT = "root";
|
||||
|
@ -161,6 +162,7 @@ public class YarnClientImpl extends YarnClient {
|
|||
timelineServiceEnabled = true;
|
||||
timelineClient = TimelineClient.createTimelineClient();
|
||||
timelineClient.init(conf);
|
||||
timelineDTRenewer = getTimelineDelegationTokenRenewer(conf);
|
||||
timelineService = TimelineUtils.buildTimelineTokenService(conf);
|
||||
}
|
||||
super.serviceInit(conf);
|
||||
|
@ -320,14 +322,22 @@ public class YarnClientImpl extends YarnClient {
|
|||
@VisibleForTesting
|
||||
org.apache.hadoop.security.token.Token<TimelineDelegationTokenIdentifier>
|
||||
getTimelineDelegationToken() throws IOException, YarnException {
|
||||
return timelineClient.getDelegationToken(timelineDTRenewer);
|
||||
}
|
||||
|
||||
private static String getTimelineDelegationTokenRenewer(Configuration conf)
|
||||
throws IOException, YarnException {
|
||||
// Parse the RM daemon user if it exists in the config
|
||||
String rmPrincipal = getConfig().get(YarnConfiguration.RM_PRINCIPAL);
|
||||
String rmPrincipal = conf.get(YarnConfiguration.RM_PRINCIPAL);
|
||||
String renewer = null;
|
||||
if (rmPrincipal != null && rmPrincipal.length() > 0) {
|
||||
HadoopKerberosName renewerKrbName = new HadoopKerberosName(rmPrincipal);
|
||||
renewer = renewerKrbName.getShortName();
|
||||
String rmHost = conf.getSocketAddr(
|
||||
YarnConfiguration.RM_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_RM_ADDRESS,
|
||||
YarnConfiguration.DEFAULT_RM_PORT).getHostName();
|
||||
renewer = SecurityUtil.getServerPrincipal(rmPrincipal, rmHost);
|
||||
}
|
||||
return timelineClient.getDelegationToken(renewer);
|
||||
return renewer;
|
||||
}
|
||||
|
||||
@Private
|
||||
|
|
|
@ -852,7 +852,25 @@ public class TestYarnClient {
|
|||
client.stop();
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Test
|
||||
public void testParseTimelineDelegationTokenRenewer() throws Exception {
|
||||
// Client side
|
||||
YarnClientImpl client = (YarnClientImpl) YarnClient.createYarnClient();
|
||||
Configuration conf = new YarnConfiguration();
|
||||
conf.setBoolean(YarnConfiguration.TIMELINE_SERVICE_ENABLED, true);
|
||||
conf.set(YarnConfiguration.RM_PRINCIPAL, "rm/_HOST@EXAMPLE.COM");
|
||||
conf.set(
|
||||
YarnConfiguration.RM_ADDRESS, "localhost:8188");
|
||||
try {
|
||||
client.init(conf);
|
||||
client.start();
|
||||
Assert.assertEquals("rm/localhost@EXAMPLE.COM", client.timelineDTRenewer);
|
||||
} finally {
|
||||
client.stop();
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testReservationAPIs() {
|
||||
// initialize
|
||||
|
|
|
@ -19,14 +19,18 @@ package org.apache.hadoop.yarn.security;
|
|||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
|
||||
import org.apache.hadoop.io.DataInputBuffer;
|
||||
import org.apache.hadoop.io.Text;
|
||||
import org.apache.hadoop.security.HadoopKerberosName;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
|
||||
import org.apache.hadoop.yarn.api.records.ApplicationId;
|
||||
import org.apache.hadoop.yarn.api.records.ContainerId;
|
||||
import org.apache.hadoop.yarn.api.records.NodeId;
|
||||
import org.apache.hadoop.yarn.api.records.Priority;
|
||||
import org.apache.hadoop.yarn.api.records.Resource;
|
||||
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
||||
import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
|
||||
import org.apache.hadoop.yarn.security.client.TimelineDelegationTokenIdentifier;
|
||||
|
@ -299,4 +303,19 @@ public class TestYARNTokenIdentifier {
|
|||
anotherToken.getMasterKeyId(), masterKeyId);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testParseTimelineDelegationTokenIdentifierRenewer() throws IOException {
|
||||
// Server side when generation a timeline DT
|
||||
Configuration conf = new YarnConfiguration();
|
||||
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTH_TO_LOCAL,
|
||||
"RULE:[2:$1@$0]([nr]m@.*EXAMPLE.COM)s/.*/yarn/");
|
||||
HadoopKerberosName.setConfiguration(conf);
|
||||
Text owner = new Text("owner");
|
||||
Text renewer = new Text("rm/localhost@EXAMPLE.COM");
|
||||
Text realUser = new Text("realUser");
|
||||
TimelineDelegationTokenIdentifier token =
|
||||
new TimelineDelegationTokenIdentifier(owner, renewer, realUser);
|
||||
Assert.assertEquals(new Text("yarn"), token.getRenewer());
|
||||
}
|
||||
|
||||
}
|
||||
|
|
Loading…
Reference in New Issue