YARN-694. Starting to use NMTokens to authenticate all communication with NodeManagers. Contributed by Omkar Vinit Joshi.

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


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1494370 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-06-18 23:20:34 +00:00
parent 8e81e6fcaf
commit 368c7ae735
51 changed files with 1617 additions and 1197 deletions

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager; import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.Clock;
@ -61,4 +62,6 @@ public interface AppContext {
Set<String> getBlacklistedNodes(); Set<String> getBlacklistedNodes();
ClientToAMTokenSecretManager getClientToAMTokenSecretManager(); ClientToAMTokenSecretManager getClientToAMTokenSecretManager();
Map<String, Token> getNMTokens();
} }

View File

@ -886,6 +886,8 @@ public class MRAppMaster extends CompositeService {
private final Configuration conf; private final Configuration conf;
private final ClusterInfo clusterInfo = new ClusterInfo(); private final ClusterInfo clusterInfo = new ClusterInfo();
private final ClientToAMTokenSecretManager clientToAMTokenSecretManager; private final ClientToAMTokenSecretManager clientToAMTokenSecretManager;
private final ConcurrentHashMap<String, org.apache.hadoop.yarn.api.records.Token> nmTokens =
new ConcurrentHashMap<String, org.apache.hadoop.yarn.api.records.Token>();
public RunningAppContext(Configuration config) { public RunningAppContext(Configuration config) {
this.conf = config; this.conf = config;
@ -952,6 +954,11 @@ public class MRAppMaster extends CompositeService {
public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() { public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
return clientToAMTokenSecretManager; return clientToAMTokenSecretManager;
} }
@Override
public Map<String, org.apache.hadoop.yarn.api.records.Token> getNMTokens() {
return this.nmTokens;
}
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")

View File

@ -19,9 +19,7 @@
package org.apache.hadoop.mapreduce.v2.app.launcher; package org.apache.hadoop.mapreduce.v2.app.launcher;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.security.PrivilegedAction;
import java.util.HashSet; import java.util.HashSet;
import java.util.Set; import java.util.Set;
import java.util.concurrent.BlockingQueue; import java.util.concurrent.BlockingQueue;
@ -35,7 +33,6 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.mapred.ShuffleHandler; import org.apache.hadoop.mapred.ShuffleHandler;
import org.apache.hadoop.mapreduce.MRJobConfig; import org.apache.hadoop.mapreduce.MRJobConfig;
import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId; import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
@ -55,6 +52,8 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@ -73,22 +72,22 @@ public class ContainerLauncherImpl extends AbstractService implements
private ConcurrentHashMap<ContainerId, Container> containers = private ConcurrentHashMap<ContainerId, Container> containers =
new ConcurrentHashMap<ContainerId, Container>(); new ConcurrentHashMap<ContainerId, Container>();
private AppContext context; private final AppContext context;
protected ThreadPoolExecutor launcherPool; protected ThreadPoolExecutor launcherPool;
protected static final int INITIAL_POOL_SIZE = 10; protected static final int INITIAL_POOL_SIZE = 10;
private int limitOnPoolSize; private int limitOnPoolSize;
private Thread eventHandlingThread; private Thread eventHandlingThread;
protected BlockingQueue<ContainerLauncherEvent> eventQueue = protected BlockingQueue<ContainerLauncherEvent> eventQueue =
new LinkedBlockingQueue<ContainerLauncherEvent>(); new LinkedBlockingQueue<ContainerLauncherEvent>();
YarnRPC rpc;
private final AtomicBoolean stopped; private final AtomicBoolean stopped;
private ContainerManagementProtocolProxy cmProxy;
private Container getContainer(ContainerLauncherEvent event) { private Container getContainer(ContainerLauncherEvent event) {
ContainerId id = event.getContainerID(); ContainerId id = event.getContainerID();
Container c = containers.get(id); Container c = containers.get(id);
if(c == null) { if(c == null) {
c = new Container(event.getTaskAttemptID(), event.getContainerID(), c = new Container(event.getTaskAttemptID(), event.getContainerID(),
event.getContainerMgrAddress(), event.getContainerToken()); event.getContainerMgrAddress());
Container old = containers.putIfAbsent(id, c); Container old = containers.putIfAbsent(id, c);
if(old != null) { if(old != null) {
c = old; c = old;
@ -114,16 +113,13 @@ public class ContainerLauncherImpl extends AbstractService implements
private TaskAttemptId taskAttemptID; private TaskAttemptId taskAttemptID;
private ContainerId containerID; private ContainerId containerID;
final private String containerMgrAddress; final private String containerMgrAddress;
private org.apache.hadoop.yarn.api.records.Token containerToken;
public Container(TaskAttemptId taId, ContainerId containerID, public Container(TaskAttemptId taId, ContainerId containerID,
String containerMgrAddress, String containerMgrAddress) {
org.apache.hadoop.yarn.api.records.Token containerToken) {
this.state = ContainerState.PREP; this.state = ContainerState.PREP;
this.taskAttemptID = taId; this.taskAttemptID = taId;
this.containerMgrAddress = containerMgrAddress; this.containerMgrAddress = containerMgrAddress;
this.containerID = containerID; this.containerID = containerID;
this.containerToken = containerToken;
} }
public synchronized boolean isCompletelyDone() { public synchronized boolean isCompletelyDone() {
@ -140,11 +136,10 @@ public class ContainerLauncherImpl extends AbstractService implements
return; return;
} }
ContainerManagementProtocol proxy = null; ContainerManagementProtocolProxyData proxy = null;
try { try {
proxy = getCMProxy(containerID, containerMgrAddress, proxy = getCMProxy(containerMgrAddress, containerID);
containerToken);
// Construct the actual Container // Construct the actual Container
ContainerLaunchContext containerLaunchContext = ContainerLaunchContext containerLaunchContext =
@ -155,7 +150,8 @@ public class ContainerLauncherImpl extends AbstractService implements
.newRecord(StartContainerRequest.class); .newRecord(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(event.getContainerToken()); startRequest.setContainerToken(event.getContainerToken());
StartContainerResponse response = proxy.startContainer(startRequest); StartContainerResponse response =
proxy.getContainerManagementProtocol().startContainer(startRequest);
ByteBuffer portInfo = ByteBuffer portInfo =
response.getAllServicesMetaData().get( response.getAllServicesMetaData().get(
@ -185,7 +181,7 @@ public class ContainerLauncherImpl extends AbstractService implements
sendContainerLaunchFailedMsg(taskAttemptID, message); sendContainerLaunchFailedMsg(taskAttemptID, message);
} finally { } finally {
if (proxy != null) { if (proxy != null) {
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig()); cmProxy.mayBeCloseProxy(proxy);
} }
} }
} }
@ -198,29 +194,30 @@ public class ContainerLauncherImpl extends AbstractService implements
} else if (!isCompletelyDone()) { } else if (!isCompletelyDone()) {
LOG.info("KILLING " + taskAttemptID); LOG.info("KILLING " + taskAttemptID);
ContainerManagementProtocol proxy = null; ContainerManagementProtocolProxyData proxy = null;
try { try {
proxy = getCMProxy(this.containerID, this.containerMgrAddress, proxy = getCMProxy(this.containerMgrAddress, this.containerID);
this.containerToken);
// kill the remote container if already launched // kill the remote container if already launched
StopContainerRequest stopRequest = Records StopContainerRequest stopRequest = Records
.newRecord(StopContainerRequest.class); .newRecord(StopContainerRequest.class);
stopRequest.setContainerId(this.containerID); stopRequest.setContainerId(this.containerID);
proxy.stopContainer(stopRequest); proxy.getContainerManagementProtocol().stopContainer(stopRequest);
} catch (Throwable t) { } catch (Throwable t) {
// ignore the cleanup failure // ignore the cleanup failure
String message = "cleanup failed for container " String message = "cleanup failed for container "
+ this.containerID + " : " + this.containerID + " : "
+ StringUtils.stringifyException(t); + StringUtils.stringifyException(t);
context.getEventHandler().handle( context.getEventHandler()
new TaskAttemptDiagnosticsUpdateEvent(this.taskAttemptID, message)); .handle(
new TaskAttemptDiagnosticsUpdateEvent(this.taskAttemptID,
message));
LOG.warn(message); LOG.warn(message);
} finally { } finally {
if (proxy != null) { if (proxy != null) {
ContainerLauncherImpl.this.rpc.stopProxy(proxy, getConfig()); cmProxy.mayBeCloseProxy(proxy);
} }
} }
this.state = ContainerState.DONE; this.state = ContainerState.DONE;
@ -239,21 +236,14 @@ public class ContainerLauncherImpl extends AbstractService implements
} }
@Override @Override
protected void serviceInit(Configuration config) throws Exception { protected void serviceInit(Configuration conf) throws Exception {
Configuration conf = new Configuration(config);
conf.setInt(
CommonConfigurationKeysPublic.IPC_CLIENT_CONNECTION_MAXIDLETIME_KEY,
0);
this.limitOnPoolSize = conf.getInt( this.limitOnPoolSize = conf.getInt(
MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT, MRJobConfig.MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT,
MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT); MRJobConfig.DEFAULT_MR_AM_CONTAINERLAUNCHER_THREAD_COUNT_LIMIT);
LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize); LOG.info("Upper limit on the thread pool size is " + this.limitOnPoolSize);
this.rpc = createYarnRPC(conf);
super.serviceInit(conf); super.serviceInit(conf);
} cmProxy =
new ContainerManagementProtocolProxy(conf, context.getNMTokens());
protected YarnRPC createYarnRPC(Configuration conf) {
return YarnRPC.create(conf);
} }
protected void serviceStart() throws Exception { protected void serviceStart() throws Exception {
@ -348,34 +338,6 @@ public class ContainerLauncherImpl extends AbstractService implements
return new EventProcessor(event); return new EventProcessor(event);
} }
protected ContainerManagementProtocol getCMProxy(ContainerId containerID,
final String containerManagerBindAddr,
org.apache.hadoop.yarn.api.records.Token containerToken)
throws IOException {
final InetSocketAddress cmAddr =
NetUtils.createSocketAddr(containerManagerBindAddr);
// the user in createRemoteUser in this context has to be ContainerID
UserGroupInformation user =
UserGroupInformation.createRemoteUser(containerID.toString());
Token<ContainerTokenIdentifier> token =
ConverterUtils.convertFromYarn(containerToken, cmAddr);
user.addToken(token);
ContainerManagementProtocol proxy = user
.doAs(new PrivilegedAction<ContainerManagementProtocol>() {
@Override
public ContainerManagementProtocol run() {
return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class,
cmAddr, getConfig());
}
});
return proxy;
}
/** /**
* Setup and start the container on remote nodemanager. * Setup and start the container on remote nodemanager.
*/ */
@ -410,7 +372,7 @@ public class ContainerLauncherImpl extends AbstractService implements
removeContainerIfDone(containerID); removeContainerIfDone(containerID);
} }
} }
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
void sendContainerLaunchFailedMsg(TaskAttemptId taskAttemptID, void sendContainerLaunchFailedMsg(TaskAttemptId taskAttemptID,
String message) { String message) {
@ -430,4 +392,9 @@ public class ContainerLauncherImpl extends AbstractService implements
throw new YarnRuntimeException(e); throw new YarnRuntimeException(e);
} }
} }
public ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData getCMProxy(
String containerMgrBindAddr, ContainerId containerId) throws IOException {
return cmProxy.getProxy(containerMgrBindAddr, containerId);
}
} }

View File

@ -63,6 +63,7 @@ import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerExitStatus; import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.NodeState; import org.apache.hadoop.yarn.api.records.NodeState;
@ -584,6 +585,14 @@ public class RMContainerAllocator extends RMContainerRequestor
} }
int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0; int newHeadRoom = getAvailableResources() != null ? getAvailableResources().getMemory() : 0;
List<Container> newContainers = response.getAllocatedContainers(); List<Container> newContainers = response.getAllocatedContainers();
// Setting NMTokens
if (response.getNMTokens() != null) {
for (NMToken nmToken : response.getNMTokens()) {
getContext().getNMTokens().put(nmToken.getNodeId().toString(),
nmToken.getToken());
}
}
List<ContainerStatus> finishedContainers = response.getCompletedContainersStatuses(); List<ContainerStatus> finishedContainers = response.getCompletedContainersStatuses();
if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) { if (newContainers.size() + finishedContainers.size() > 0 || headRoom != newHeadRoom) {
//something changed //something changed

View File

@ -26,6 +26,7 @@ import org.apache.hadoop.mapreduce.v2.api.records.JobId;
import org.apache.hadoop.mapreduce.v2.app.job.Job; import org.apache.hadoop.mapreduce.v2.app.job.Job;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.util.Clock; import org.apache.hadoop.yarn.util.Clock;
import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager; import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
@ -130,4 +131,10 @@ public class MockAppContext implements AppContext {
// Not implemented // Not implemented
return null; return null;
} }
@Override
public Map<String, Token> getNMTokens() {
// Not Implemented
return null;
}
} }

View File

@ -47,6 +47,7 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
import org.junit.Test; import org.junit.Test;
/** /**
@ -225,8 +226,8 @@ public class TestFail {
} }
@Override @Override
protected ContainerManagementProtocol getCMProxy(ContainerId contianerID, public ContainerManagementProtocolProxyData getCMProxy(
String containerManagerBindAddr, Token containerToken) String containerMgrBindAddr, ContainerId containerId)
throws IOException { throws IOException {
try { try {
synchronized (this) { synchronized (this) {

View File

@ -68,6 +68,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
@ -857,7 +858,13 @@ public class TestRuntimeEstimators {
return null; return null;
} }
@Override
public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() { public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
return null;
}
@Override
public Map<String, Token> getNMTokens() {
// Not Implemented // Not Implemented
return null; return null;
} }

View File

@ -62,12 +62,15 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC; import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.junit.Test; import org.junit.Test;
public class TestContainerLauncher { public class TestContainerLauncher {
@ -342,16 +345,26 @@ public class TestContainerLauncher {
} }
@Override @Override
protected ContainerLauncher createContainerLauncher(AppContext context) { protected ContainerLauncher
createContainerLauncher(final AppContext context) {
return new ContainerLauncherImpl(context) { return new ContainerLauncherImpl(context) {
@Override @Override
protected ContainerManagementProtocol getCMProxy(ContainerId containerID, public ContainerManagementProtocolProxyData getCMProxy(
String containerManagerBindAddr, Token containerToken) String containerMgrBindAddr, ContainerId containerId)
throws IOException { throws IOException {
// make proxy connect to our local containerManager server Token dummyToken =
ContainerManagementProtocol proxy = (ContainerManagementProtocol) rpc.getProxy( Token.newInstance("NMTokenIdentifier".getBytes(),
ContainerManagementProtocol.class, NMTokenIdentifier.KIND.toString(), "password".getBytes(),
NetUtils.getConnectAddress(server), conf); "NMToken");
ContainerManagementProtocolProxy cmProxy =
new ContainerManagementProtocolProxy(conf, context.getNMTokens());
InetSocketAddress addr = NetUtils.getConnectAddress(server);
ContainerManagementProtocolProxyData proxy =
cmProxy.new ContainerManagementProtocolProxyData(
YarnRPC.create(conf),
addr.getHostName() + ":" + addr.getPort(), containerId,
dummyToken);
return proxy; return proxy;
} }
}; };

View File

@ -95,11 +95,6 @@ public class TestContainerLauncherImpl {
this.rpc = rpc; this.rpc = rpc;
} }
@Override
protected YarnRPC createYarnRPC(Configuration conf) {
return rpc;
}
public void waitForPoolToIdle() throws InterruptedException { public void waitForPoolToIdle() throws InterruptedException {
//I wish that we did not need the sleep, but it is here so that we are sure //I wish that we did not need the sleep, but it is here so that we are sure
// That the other thread had time to insert the event into the queue and // That the other thread had time to insert the event into the queue and

View File

@ -44,6 +44,7 @@ import org.apache.hadoop.service.Service;
import org.apache.hadoop.util.ReflectionUtils; import org.apache.hadoop.util.ReflectionUtils;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException; import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@ -315,4 +316,10 @@ public class JobHistory extends AbstractService implements HistoryContext {
// Not implemented. // Not implemented.
return null; return null;
} }
@Override
public Map<String, Token> getNMTokens() {
// Not Implemented.
return null;
}
} }

View File

@ -179,6 +179,9 @@ Release 2.1.0-beta - UNRELEASED
YARN-850. Rename getClusterAvailableResources to getAvailableResources in YARN-850. Rename getClusterAvailableResources to getAvailableResources in
AMRMClients (Jian He via bikas) AMRMClients (Jian He via bikas)
YARN-694. Starting to use NMTokens to authenticate all communication with
NodeManagers. (Omkar Vinit Joshi via vinodkv)
NEW FEATURES NEW FEATURES
YARN-482. FS: Extend SchedulingMode to intermediate queues. YARN-482. FS: Extend SchedulingMode to intermediate queues.

View File

@ -728,6 +728,23 @@ public class YarnConfiguration extends Configuration {
YARN_PREFIX + "client.nodemanager-client-async.thread-pool-max-size"; YARN_PREFIX + "client.nodemanager-client-async.thread-pool-max-size";
public static final int DEFAULT_NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE = 500; public static final int DEFAULT_NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE = 500;
/**
* Maximum number of proxy connections for node manager. It should always be
* more than 1. NMClient and MRAppMaster will use this to cache connection
* with node manager. There will be at max one connection per node manager.
* Ex. configuring it to a value of 5 will make sure that client will at
* max have 5 connections cached with 5 different node managers. These
* connections will be timed out if idle for more than system wide idle
* timeout period. The token if used for authentication then it will be used
* only at connection creation time. If new token is received then earlier
* connection should be closed in order to use newer token.
* Note: {@link YarnConfiguration#NM_CLIENT_ASYNC_THREAD_POOL_MAX_SIZE}
* are related to each other.
*/
public static final String NM_CLIENT_MAX_NM_PROXIES =
YARN_PREFIX + "client.max-nodemanagers-proxies";
public static final int DEFAULT_NM_CLIENT_MAX_NM_PROXIES = 500;
public YarnConfiguration() { public YarnConfiguration() {
super(); super();
} }

View File

@ -70,6 +70,7 @@ import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest; import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest;
import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync; import org.apache.hadoop.yarn.client.api.async.AMRMClientAsync;
import org.apache.hadoop.yarn.client.api.async.NMClientAsync; import org.apache.hadoop.yarn.client.api.async.NMClientAsync;
import org.apache.hadoop.yarn.client.api.async.impl.NMClientAsyncImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
@ -447,7 +448,8 @@ public class ApplicationMaster {
resourceManager.start(); resourceManager.start();
containerListener = new NMCallbackHandler(); containerListener = new NMCallbackHandler();
nmClientAsync = NMClientAsync.createNMClientAsync(containerListener); nmClientAsync =
new NMClientAsyncImpl(containerListener, resourceManager.getNMTokens());
nmClientAsync.init(conf); nmClientAsync.init(conf);
nmClientAsync.start(); nmClientAsync.start();
@ -683,8 +685,7 @@ public class ApplicationMaster {
} }
Container container = containers.get(containerId); Container container = containers.get(containerId);
if (container != null) { if (container != null) {
nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId(), nmClientAsync.getContainerStatusAsync(containerId, container.getNodeId());
container.getContainerToken());
} }
} }

View File

@ -26,6 +26,7 @@ import java.io.OutputStream;
import java.net.URL; import java.net.URL;
import junit.framework.Assert; import junit.framework.Assert;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
@ -33,6 +34,8 @@ import org.apache.hadoop.util.JarFinder;
import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.server.MiniYARNCluster; import org.apache.hadoop.yarn.server.MiniYARNCluster;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler; 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.scheduler.fifo.FifoScheduler;
import org.junit.AfterClass; import org.junit.AfterClass;
@ -50,7 +53,7 @@ public class TestDistributedShell {
protected static String APPMASTER_JAR = JarFinder.getJar(ApplicationMaster.class); protected static String APPMASTER_JAR = JarFinder.getJar(ApplicationMaster.class);
@BeforeClass @BeforeClass
public static void setup() throws InterruptedException, IOException { public static void setup() throws InterruptedException, Exception {
LOG.info("Starting up YARN cluster"); LOG.info("Starting up YARN cluster");
conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128); conf.setInt(YarnConfiguration.RM_SCHEDULER_MINIMUM_ALLOCATION_MB, 128);
conf.setClass(YarnConfiguration.RM_SCHEDULER, conf.setClass(YarnConfiguration.RM_SCHEDULER,
@ -60,6 +63,9 @@ public class TestDistributedShell {
TestDistributedShell.class.getSimpleName(), 1, 1, 1); TestDistributedShell.class.getSimpleName(), 1, 1, 1);
yarnCluster.init(conf); yarnCluster.init(conf);
yarnCluster.start(); yarnCluster.start();
NodeManager nm = yarnCluster.getNodeManager(0);
waitForNMToRegister(nm);
URL url = Thread.currentThread().getContextClassLoader().getResource("yarn-site.xml"); URL url = Thread.currentThread().getContextClassLoader().getResource("yarn-site.xml");
if (url == null) { if (url == null) {
throw new RuntimeException("Could not find 'yarn-site.xml' dummy file in classpath"); throw new RuntimeException("Could not find 'yarn-site.xml' dummy file in classpath");
@ -195,5 +201,14 @@ public class TestDistributedShell {
} }
} }
protected static void waitForNMToRegister(NodeManager nm)
throws Exception {
int attempt = 60;
ContainerManagerImpl cm =
((ContainerManagerImpl) nm.getNMContext().getContainerManager());
while (cm.getBlockNewContainerRequestsStatus() && attempt-- > 0) {
Thread.sleep(2000);
}
}
} }

View File

@ -1,4 +1,5 @@
/** /**
* Licensed to the Apache Software Foundation (ASF) under one * Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
@ -21,16 +22,19 @@ package org.apache.hadoop.yarn.client.api;
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceAudience;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.classification.InterfaceStability;
import org.apache.hadoop.service.AbstractService; import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.client.api.impl.NMClientImpl; import org.apache.hadoop.yarn.client.api.impl.NMClientImpl;
@ -42,19 +46,30 @@ public abstract class NMClient extends AbstractService {
/** /**
* Create a new instance of NMClient. * Create a new instance of NMClient.
* @param nmTokens need to pass map of NMTokens which are received on
* {@link AMRMClient#allocate(float)} call as a part of
* {@link AllocateResponse}.
* key :- NodeAddr (host:port)
* Value :- Token {@link NMToken#getToken()}
*/ */
@Public @Public
public static NMClient createNMClient() { public static NMClient createNMClient(ConcurrentMap<String, Token> nmTokens) {
NMClient client = new NMClientImpl(); NMClient client = new NMClientImpl(nmTokens);
return client; return client;
} }
/** /**
* Create a new instance of NMClient. * Create a new instance of NMClient.
* @param nmTokens need to pass map of NMTokens which are received on
* {@link AMRMClient#allocate(float)} call as a part of
* {@link AllocateResponse}.
* key :- NodeAddr (host:port)
* Value :- Token {@link NMToken#getToken()}
*/ */
@Public @Public
public static NMClient createNMClient(String name) { public static NMClient createNMClient(String name,
NMClient client = new NMClientImpl(name); ConcurrentMap<String, Token> nmTokens) {
NMClient client = new NMClientImpl(name, nmTokens);
return client; return client;
} }
@ -89,35 +104,33 @@ public abstract class NMClient extends AbstractService {
* *
* @param containerId the Id of the started container * @param containerId the Id of the started container
* @param nodeId the Id of the <code>NodeManager</code> * @param nodeId the Id of the <code>NodeManager</code>
* @param containerToken the security token to verify authenticity of the *
* started container
* @throws YarnException * @throws YarnException
* @throws IOException * @throws IOException
*/ */
public abstract void stopContainer(ContainerId containerId, NodeId nodeId, public abstract void stopContainer(ContainerId containerId, NodeId nodeId)
Token containerToken) throws YarnException, IOException; throws YarnException, IOException;
/** /**
* <p>Query the status of a container.</p> * <p>Query the status of a container.</p>
* *
* @param containerId the Id of the started container * @param containerId the Id of the started container
* @param nodeId the Id of the <code>NodeManager</code> * @param nodeId the Id of the <code>NodeManager</code>
* @param containerToken the security token to verify authenticity of the *
* started container
* @return the status of a container * @return the status of a container
* @throws YarnException * @throws YarnException
* @throws IOException * @throws IOException
*/ */
public abstract ContainerStatus getContainerStatus(ContainerId containerId, NodeId nodeId, public abstract ContainerStatus getContainerStatus(ContainerId containerId,
Token containerToken) throws YarnException, IOException; NodeId nodeId) throws YarnException, IOException;
/** /**
* <p>Set whether the containers that are started by this client, and are * <p>Set whether the containers that are started by this client, and are
* still running should be stopped when the client stops. By default, the * still running should be stopped when the client stops. By default, the
* feature should be enabled.</p> * feature should be enabled.</p> However, containers will be stopped only
* when service is stopped. i.e. after {@link NMClient#stop()}.
* *
* @param enabled whether the feature is enabled or not * @param enabled whether the feature is enabled or not
*/ */
public abstract void cleanupRunningContainersOnStop(boolean enabled); public abstract void cleanupRunningContainersOnStop(boolean enabled);
} }

View File

@ -20,6 +20,7 @@ package org.apache.hadoop.yarn.client.api.async;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentMap;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
@ -110,16 +111,19 @@ public abstract class NMClientAsync extends AbstractService {
protected NMClient client; protected NMClient client;
protected CallbackHandler callbackHandler; protected CallbackHandler callbackHandler;
public static NMClientAsync createNMClientAsync(CallbackHandler callbackHandler) { public static NMClientAsync createNMClientAsync(
return new NMClientAsyncImpl(callbackHandler); CallbackHandler callbackHandler, ConcurrentMap<String, Token> nmTokens) {
return new NMClientAsyncImpl(callbackHandler, nmTokens);
} }
protected NMClientAsync(CallbackHandler callbackHandler) { protected NMClientAsync(CallbackHandler callbackHandler,
this (NMClientAsync.class.getName(), callbackHandler); ConcurrentMap<String, Token> nmTokens) {
this (NMClientAsync.class.getName(), callbackHandler, nmTokens);
} }
protected NMClientAsync(String name, CallbackHandler callbackHandler) { protected NMClientAsync(String name, CallbackHandler callbackHandler,
this (name, new NMClientImpl(), callbackHandler); ConcurrentMap<String, Token> nmTokens) {
this (name, new NMClientImpl(nmTokens), callbackHandler);
} }
@Private @Private
@ -135,10 +139,10 @@ public abstract class NMClientAsync extends AbstractService {
Container container, ContainerLaunchContext containerLaunchContext); Container container, ContainerLaunchContext containerLaunchContext);
public abstract void stopContainerAsync( public abstract void stopContainerAsync(
ContainerId containerId, NodeId nodeId, Token containerToken); ContainerId containerId, NodeId nodeId);
public abstract void getContainerStatusAsync( public abstract void getContainerStatusAsync(
ContainerId containerId, NodeId nodeId, Token containerToken); ContainerId containerId, NodeId nodeId);
public NMClient getClient() { public NMClient getClient() {
return client; return client;

View File

@ -82,12 +82,14 @@ public class NMClientAsyncImpl extends NMClientAsync {
protected ConcurrentMap<ContainerId, StatefulContainer> containers = protected ConcurrentMap<ContainerId, StatefulContainer> containers =
new ConcurrentHashMap<ContainerId, StatefulContainer>(); new ConcurrentHashMap<ContainerId, StatefulContainer>();
public NMClientAsyncImpl(CallbackHandler callbackHandler) { public NMClientAsyncImpl(CallbackHandler callbackHandler,
this (NMClientAsyncImpl.class.getName(), callbackHandler); ConcurrentMap<String, Token> nmTokens) {
this(NMClientAsync.class.getName(), callbackHandler, nmTokens);
} }
public NMClientAsyncImpl(String name, CallbackHandler callbackHandler) { public NMClientAsyncImpl(String name, CallbackHandler callbackHandler,
this (name, new NMClientImpl(), callbackHandler); ConcurrentMap<String, Token> nmTokens) {
this(name, new NMClientImpl(nmTokens), callbackHandler);
} }
@Private @Private
@ -229,15 +231,14 @@ public class NMClientAsyncImpl extends NMClientAsync {
} }
} }
public void stopContainerAsync(ContainerId containerId, NodeId nodeId, public void stopContainerAsync(ContainerId containerId, NodeId nodeId) {
Token containerToken) {
if (containers.get(containerId) == null) { if (containers.get(containerId) == null) {
callbackHandler.onStopContainerError(containerId, callbackHandler.onStopContainerError(containerId,
RPCUtil.getRemoteException("Container " + containerId + RPCUtil.getRemoteException("Container " + containerId +
" is neither started nor scheduled to start")); " is neither started nor scheduled to start"));
} }
try { try {
events.put(new ContainerEvent(containerId, nodeId, containerToken, events.put(new ContainerEvent(containerId, nodeId, null,
ContainerEventType.STOP_CONTAINER)); ContainerEventType.STOP_CONTAINER));
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.warn("Exception when scheduling the event of stopping Container " + LOG.warn("Exception when scheduling the event of stopping Container " +
@ -246,10 +247,9 @@ public class NMClientAsyncImpl extends NMClientAsync {
} }
} }
public void getContainerStatusAsync(ContainerId containerId, NodeId nodeId, public void getContainerStatusAsync(ContainerId containerId, NodeId nodeId) {
Token containerToken) {
try { try {
events.put(new ContainerEvent(containerId, nodeId, containerToken, events.put(new ContainerEvent(containerId, nodeId, null,
ContainerEventType.QUERY_CONTAINER)); ContainerEventType.QUERY_CONTAINER));
} catch (InterruptedException e) { } catch (InterruptedException e) {
LOG.warn("Exception when scheduling the event of querying the status" + LOG.warn("Exception when scheduling the event of querying the status" +
@ -421,9 +421,9 @@ public class NMClientAsyncImpl extends NMClientAsync {
StatefulContainer container, ContainerEvent event) { StatefulContainer container, ContainerEvent event) {
ContainerId containerId = event.getContainerId(); ContainerId containerId = event.getContainerId();
try { try {
container.nmClientAsync.getClient().stopContainer( container.nmClientAsync.getClient().stopContainer(
containerId, event.getNodeId(), event.getContainerToken()); containerId, event.getNodeId());
try { try {
container.nmClientAsync.getCallbackHandler().onContainerStopped( container.nmClientAsync.getCallbackHandler().onContainerStopped(
event.getContainerId()); event.getContainerId());
} catch (Throwable thr) { } catch (Throwable thr) {
@ -534,7 +534,7 @@ public class NMClientAsyncImpl extends NMClientAsync {
if (event.getType() == ContainerEventType.QUERY_CONTAINER) { if (event.getType() == ContainerEventType.QUERY_CONTAINER) {
try { try {
ContainerStatus containerStatus = client.getContainerStatus( ContainerStatus containerStatus = client.getContainerStatus(
containerId, event.getNodeId(), event.getContainerToken()); containerId, event.getNodeId());
try { try {
callbackHandler.onContainerStatusReceived( callbackHandler.onContainerStatusReceived(
containerId, containerStatus); containerId, containerStatus);

View File

@ -0,0 +1,237 @@
/**
* 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.client.api.impl;
import java.net.InetSocketAddress;
import java.security.PrivilegedAction;
import java.util.ArrayList;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.LimitedPrivate;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.util.ConverterUtils;
import com.google.common.annotations.VisibleForTesting;
/**
* Helper class to manage container manager proxies
*/
@LimitedPrivate({ "MapReduce", "YARN" })
public class ContainerManagementProtocolProxy {
static final Log LOG = LogFactory.getLog(ContainerManagementProtocolProxy.class);
private final int maxConnectedNMs;
private final LinkedHashMap<String, ContainerManagementProtocolProxyData> cmProxy;
private Map<String, Token> nmTokens;
private final Configuration conf;
private final YarnRPC rpc;
public ContainerManagementProtocolProxy(Configuration conf,
Map<String, Token> nmTokens) {
this.nmTokens = nmTokens;
this.conf = conf;
maxConnectedNMs =
conf.getInt(YarnConfiguration.NM_CLIENT_MAX_NM_PROXIES,
YarnConfiguration.DEFAULT_NM_CLIENT_MAX_NM_PROXIES);
if (maxConnectedNMs < 1) {
throw new YarnRuntimeException(
YarnConfiguration.NM_CLIENT_MAX_NM_PROXIES
+ " (" + maxConnectedNMs + ") can not be less than 1.");
}
LOG.info(YarnConfiguration.NM_CLIENT_MAX_NM_PROXIES + " : "
+ maxConnectedNMs);
cmProxy =
new LinkedHashMap<String, ContainerManagementProtocolProxyData>();
rpc = YarnRPC.create(conf);
}
public synchronized ContainerManagementProtocolProxyData getProxy(
String containerManagerBindAddr, ContainerId containerId)
throws InvalidToken {
// This get call will update the map which is working as LRU cache.
ContainerManagementProtocolProxyData proxy =
cmProxy.get(containerManagerBindAddr);
while (proxy != null
&& !proxy.token.getIdentifier().equals(
nmTokens.get(containerManagerBindAddr).getIdentifier())) {
LOG.info("Refreshing proxy as NMToken got updated for node : "
+ containerManagerBindAddr);
// Token is updated. check if anyone has already tried closing it.
if (!proxy.scheduledForClose) {
// try closing the proxy. Here if someone is already using it
// then we might not close it. In which case we will wait.
removeProxy(proxy);
} else {
try {
this.wait();
} catch (InterruptedException e) {
e.printStackTrace();
}
}
if (proxy.activeCallers < 0) {
proxy = cmProxy.get(containerManagerBindAddr);
}
}
if (proxy == null) {
proxy =
new ContainerManagementProtocolProxyData(rpc, containerManagerBindAddr,
containerId, nmTokens.get(containerManagerBindAddr));
if (cmProxy.size() > maxConnectedNMs) {
// Number of existing proxy exceed the limit.
String cmAddr = cmProxy.keySet().iterator().next();
removeProxy(cmProxy.get(cmAddr));
}
cmProxy.put(containerManagerBindAddr, proxy);
}
// This is to track active users of this proxy.
proxy.activeCallers++;
updateLRUCache(containerManagerBindAddr);
return proxy;
}
private void updateLRUCache(String containerManagerBindAddr) {
ContainerManagementProtocolProxyData proxy =
cmProxy.remove(containerManagerBindAddr);
cmProxy.put(containerManagerBindAddr, proxy);
}
public synchronized void mayBeCloseProxy(
ContainerManagementProtocolProxyData proxy) {
proxy.activeCallers--;
if (proxy.scheduledForClose && proxy.activeCallers < 0) {
LOG.info("Closing proxy : " + proxy.containerManagerBindAddr);
cmProxy.remove(proxy.containerManagerBindAddr);
try {
rpc.stopProxy(proxy.getContainerManagementProtocol(), conf);
} finally {
this.notifyAll();
}
}
}
private synchronized void removeProxy(
ContainerManagementProtocolProxyData proxy) {
if (!proxy.scheduledForClose) {
proxy.scheduledForClose = true;
mayBeCloseProxy(proxy);
}
}
public synchronized void stopAllProxies() {
List<String> nodeIds = new ArrayList<String>();
nodeIds.addAll(this.cmProxy.keySet());
for (String nodeId : nodeIds) {
ContainerManagementProtocolProxyData proxy = cmProxy.get(nodeId);
// Explicitly reducing the proxy count to allow stopping proxy.
proxy.activeCallers = 0;
try {
removeProxy(proxy);
} catch (Throwable t) {
LOG.error("Error closing connection", t);
}
}
cmProxy.clear();
}
public synchronized void setNMTokens(Map<String, Token> nmTokens) {
this.nmTokens = nmTokens;
}
public class ContainerManagementProtocolProxyData {
private final String containerManagerBindAddr;
private final ContainerManagementProtocol proxy;
private int activeCallers;
private boolean scheduledForClose;
private final Token token;
@Private
@VisibleForTesting
public ContainerManagementProtocolProxyData(YarnRPC rpc,
String containerManagerBindAddr,
ContainerId containerId, Token token) throws InvalidToken {
this.containerManagerBindAddr = containerManagerBindAddr;
;
this.activeCallers = 0;
this.scheduledForClose = false;
this.token = token;
this.proxy = newProxy(rpc, containerManagerBindAddr, containerId, token);
}
@Private
@VisibleForTesting
protected ContainerManagementProtocol newProxy(final YarnRPC rpc,
String containerManagerBindAddr, ContainerId containerId, Token token)
throws InvalidToken {
if (token == null) {
throw new InvalidToken("No NMToken sent for "
+ containerManagerBindAddr);
}
final InetSocketAddress cmAddr =
NetUtils.createSocketAddr(containerManagerBindAddr);
LOG.info("Opening proxy : " + containerManagerBindAddr);
// the user in createRemoteUser in this context has to be ContainerID
UserGroupInformation user =
UserGroupInformation.createRemoteUser(containerId
.getApplicationAttemptId().toString());
org.apache.hadoop.security.token.Token<NMTokenIdentifier> nmToken =
ConverterUtils.convertFromYarn(token, cmAddr);
user.addToken(nmToken);
ContainerManagementProtocol proxy = user
.doAs(new PrivilegedAction<ContainerManagementProtocol>() {
@Override
public ContainerManagementProtocol run() {
return (ContainerManagementProtocol) rpc.getProxy(
ContainerManagementProtocol.class, cmAddr, conf);
}
});
return proxy;
}
public ContainerManagementProtocol getContainerManagementProtocol() {
return proxy;
}
}
}

View File

@ -19,9 +19,7 @@
package org.apache.hadoop.yarn.client.api.impl; package org.apache.hadoop.yarn.client.api.impl;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.security.PrivilegedAction;
import java.util.Map; import java.util.Map;
import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.ConcurrentMap; import java.util.concurrent.ConcurrentMap;
@ -29,31 +27,23 @@ import java.util.concurrent.atomic.AtomicBoolean;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.AbstractService;
import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.NMClient;
import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records;
/** /**
* <p> * <p>
@ -91,14 +81,18 @@ public class NMClientImpl extends NMClient {
new ConcurrentHashMap<ContainerId, StartedContainer>(); new ConcurrentHashMap<ContainerId, StartedContainer>();
//enabled by default //enabled by default
private final AtomicBoolean cleanupRunningContainers = new AtomicBoolean(true); private final AtomicBoolean cleanupRunningContainers = new AtomicBoolean(true);
private ContainerManagementProtocolProxy cmProxy;
private ConcurrentMap<String, Token> nmTokens;
public NMClientImpl() { public NMClientImpl(ConcurrentMap<String, Token> nmTokens) {
super(NMClientImpl.class.getName()); super(NMClientImpl.class.getName());
this.nmTokens = nmTokens;
} }
public NMClientImpl(String name) { public NMClientImpl(String name, ConcurrentMap<String, Token> nmTokens) {
super(name); super(name);
this.nmTokens = nmTokens;
} }
@Override @Override
@ -108,6 +102,7 @@ public class NMClientImpl extends NMClient {
if (getCleanupRunningContainers().get()) { if (getCleanupRunningContainers().get()) {
cleanupRunningContainers(); cleanupRunningContainers();
} }
cmProxy.stopAllProxies();
super.serviceStop(); super.serviceStop();
} }
@ -115,8 +110,7 @@ public class NMClientImpl extends NMClient {
for (StartedContainer startedContainer : startedContainers.values()) { for (StartedContainer startedContainer : startedContainers.values()) {
try { try {
stopContainer(startedContainer.getContainerId(), stopContainer(startedContainer.getContainerId(),
startedContainer.getNodeId(), startedContainer.getNodeId());
startedContainer.getContainerToken());
} catch (YarnException e) { } catch (YarnException e) {
LOG.error("Failed to stop Container " + LOG.error("Failed to stop Container " +
startedContainer.getContainerId() + startedContainer.getContainerId() +
@ -129,23 +123,29 @@ public class NMClientImpl extends NMClient {
} }
} }
@Override
protected void serviceInit(Configuration conf) throws Exception {
super.serviceInit(conf);
cmProxy =
new ContainerManagementProtocolProxy(conf, nmTokens);
}
@Override @Override
public void cleanupRunningContainersOnStop(boolean enabled) { public void cleanupRunningContainersOnStop(boolean enabled) {
getCleanupRunningContainers().set(enabled); getCleanupRunningContainers().set(enabled);
} }
protected static class StartedContainer { protected static class StartedContainer {
private ContainerId containerId; private ContainerId containerId;
private NodeId nodeId; private NodeId nodeId;
private Token containerToken; private ContainerState state;
private boolean stopped;
public StartedContainer(ContainerId containerId, NodeId nodeId, public StartedContainer(ContainerId containerId, NodeId nodeId,
Token containerToken) { Token containerToken) {
this.containerId = containerId; this.containerId = containerId;
this.nodeId = nodeId; this.nodeId = nodeId;
this.containerToken = containerToken; state = ContainerState.NEW;
stopped = false;
} }
public ContainerId getContainerId() { public ContainerId getContainerId() {
@ -155,137 +155,17 @@ public class NMClientImpl extends NMClient {
public NodeId getNodeId() { public NodeId getNodeId() {
return nodeId; return nodeId;
} }
public Token getContainerToken() {
return containerToken;
}
} }
protected static final class NMCommunicator extends AbstractService { private void addStartingContainer(StartedContainer startedContainer)
private ContainerId containerId; throws YarnException {
private NodeId nodeId; if (startedContainers.putIfAbsent(startedContainer.containerId,
private Token containerToken; startedContainer) != null) {
private ContainerManagementProtocol containerManager; throw RPCUtil.getRemoteException("Container "
+ startedContainer.containerId.toString() + " is already started");
public NMCommunicator(ContainerId containerId, NodeId nodeId,
Token containerToken) {
super(NMCommunicator.class.getName());
this.containerId = containerId;
this.nodeId = nodeId;
this.containerToken = containerToken;
}
@Override
protected synchronized void serviceStart() throws Exception {
final YarnRPC rpc = YarnRPC.create(getConfig());
final InetSocketAddress containerAddress =
NetUtils.createSocketAddr(nodeId.toString());
// the user in createRemoteUser in this context has to be ContainerId
UserGroupInformation currentUser =
UserGroupInformation.createRemoteUser(containerId.toString());
org.apache.hadoop.security.token.Token<ContainerTokenIdentifier> token =
ConverterUtils.convertFromYarn(containerToken, containerAddress);
currentUser.addToken(token);
containerManager = currentUser
.doAs(new PrivilegedAction<ContainerManagementProtocol>() {
@Override
public ContainerManagementProtocol run() {
return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class,
containerAddress, getConfig());
}
});
LOG.debug("Connecting to ContainerManager at " + containerAddress);
super.serviceStart();
}
@Override
protected synchronized void serviceStop() throws Exception {
if (this.containerManager != null) {
RPC.stopProxy(this.containerManager);
if (LOG.isDebugEnabled()) {
InetSocketAddress containerAddress =
NetUtils.createSocketAddr(nodeId.toString());
LOG.debug("Disconnecting from ContainerManager at " +
containerAddress);
}
}
super.serviceStop();
}
public synchronized Map<String, ByteBuffer> startContainer(
Container container, ContainerLaunchContext containerLaunchContext)
throws YarnException, IOException {
if (!container.getId().equals(containerId)) {
throw new IllegalArgumentException(
"NMCommunicator's containerId mismatches the given Container's");
}
StartContainerResponse startResponse = null;
try {
StartContainerRequest startRequest =
Records.newRecord(StartContainerRequest.class);
startRequest.setContainerToken(container.getContainerToken());
startRequest.setContainerLaunchContext(containerLaunchContext);
startResponse = containerManager.startContainer(startRequest);
if (LOG.isDebugEnabled()) {
LOG.debug("Started Container " + containerId);
}
} catch (YarnException e) {
LOG.warn("Container " + containerId + " failed to start", e);
throw e;
} catch (IOException e) {
LOG.warn("Container " + containerId + " failed to start", e);
throw e;
}
return startResponse.getAllServicesMetaData();
}
public synchronized void stopContainer() throws YarnException,
IOException {
try {
StopContainerRequest stopRequest =
Records.newRecord(StopContainerRequest.class);
stopRequest.setContainerId(containerId);
containerManager.stopContainer(stopRequest);
if (LOG.isDebugEnabled()) {
LOG.debug("Stopped Container " + containerId);
}
} catch (YarnException e) {
LOG.warn("Container " + containerId + " failed to stop", e);
throw e;
} catch (IOException e) {
LOG.warn("Container " + containerId + " failed to stop", e);
throw e;
}
}
public synchronized ContainerStatus getContainerStatus()
throws YarnException, IOException {
GetContainerStatusResponse statusResponse = null;
try {
GetContainerStatusRequest statusRequest =
Records.newRecord(GetContainerStatusRequest.class);
statusRequest.setContainerId(containerId);
statusResponse = containerManager.getContainerStatus(statusRequest);
if (LOG.isDebugEnabled()) {
LOG.debug("Got the status of Container " + containerId);
}
} catch (YarnException e) {
LOG.warn(
"Unable to get the status of Container " + containerId, e);
throw e;
} catch (IOException e) {
LOG.warn(
"Unable to get the status of Container " + containerId, e);
throw e;
}
return statusResponse.getStatus();
} }
startedContainers
.put(startedContainer.getContainerId(), startedContainer);
} }
@Override @Override
@ -293,108 +173,112 @@ public class NMClientImpl extends NMClient {
Container container, ContainerLaunchContext containerLaunchContext) Container container, ContainerLaunchContext containerLaunchContext)
throws YarnException, IOException { throws YarnException, IOException {
// Do synchronization on StartedContainer to prevent race condition // Do synchronization on StartedContainer to prevent race condition
// between startContainer and stopContainer // between startContainer and stopContainer only when startContainer is
synchronized (addStartedContainer(container)) { // in progress for a given container.
StartedContainer startingContainer = createStartedContainer(container);
synchronized (startingContainer) {
addStartingContainer(startingContainer);
Map<String, ByteBuffer> allServiceResponse; Map<String, ByteBuffer> allServiceResponse;
NMCommunicator nmCommunicator = null; ContainerManagementProtocolProxyData proxy = null;
try { try {
nmCommunicator = new NMCommunicator(container.getId(), proxy =
container.getNodeId(), container.getContainerToken()); cmProxy.getProxy(container.getNodeId().toString(),
nmCommunicator.init(getConfig()); container.getId());
nmCommunicator.start();
allServiceResponse = allServiceResponse =
nmCommunicator.startContainer(container, containerLaunchContext); proxy
.getContainerManagementProtocol().startContainer(
StartContainerRequest.newInstance(containerLaunchContext,
container.getContainerToken())).getAllServicesMetaData();
startingContainer.state = ContainerState.RUNNING;
} catch (YarnException e) { } catch (YarnException e) {
startingContainer.state = ContainerState.COMPLETE;
// Remove the started container if it failed to start // Remove the started container if it failed to start
removeStartedContainer(container.getId()); removeStartedContainer(startingContainer);
throw e; throw e;
} catch (IOException e) { } catch (IOException e) {
removeStartedContainer(container.getId()); startingContainer.state = ContainerState.COMPLETE;
removeStartedContainer(startingContainer);
throw e; throw e;
} catch (Throwable t) { } catch (Throwable t) {
removeStartedContainer(container.getId()); startingContainer.state = ContainerState.COMPLETE;
removeStartedContainer(startingContainer);
throw RPCUtil.getRemoteException(t); throw RPCUtil.getRemoteException(t);
} finally { } finally {
if (nmCommunicator != null) { if (proxy != null) {
nmCommunicator.stop(); cmProxy.mayBeCloseProxy(proxy);
} }
} }
return allServiceResponse; return allServiceResponse;
} }
// Three choices:
// 1. starting and releasing the proxy before and after each interaction
// 2. starting the proxy when starting the container and releasing it when
// stopping the container
// 3. starting the proxy when starting the container and releasing it when
// stopping the client
// Adopt 1 currently
} }
@Override @Override
public void stopContainer(ContainerId containerId, NodeId nodeId, public void stopContainer(ContainerId containerId, NodeId nodeId)
Token containerToken) throws YarnException, IOException { throws YarnException, IOException {
StartedContainer startedContainer = getStartedContainer(containerId); StartedContainer startedContainer = getStartedContainer(containerId);
if (startedContainer == null) {
throw RPCUtil.getRemoteException("Container " + containerId +
" is either not started yet or already stopped");
}
// Only allow one request of stopping the container to move forward // Only allow one request of stopping the container to move forward
// When entering the block, check whether the precursor has already stopped // When entering the block, check whether the precursor has already stopped
// the container // the container
synchronized (startedContainer) { if (startedContainer != null) {
if (startedContainer.stopped) { synchronized (startedContainer) {
return; if (startedContainer.state != ContainerState.RUNNING) {
} return;
NMCommunicator nmCommunicator = null;
try {
nmCommunicator =
new NMCommunicator(containerId, nodeId, containerToken);
nmCommunicator.init(getConfig());
nmCommunicator.start();
nmCommunicator.stopContainer();
} finally {
if (nmCommunicator != null) {
nmCommunicator.stop();
} }
startedContainer.stopped = true; stopContainerInternal(containerId, nodeId);
removeStartedContainer(containerId); // Only after successful
startedContainer.state = ContainerState.COMPLETE;
removeStartedContainer(startedContainer);
} }
} else {
stopContainerInternal(containerId, nodeId);
} }
} }
@Override @Override
public ContainerStatus getContainerStatus(ContainerId containerId, public ContainerStatus getContainerStatus(ContainerId containerId,
NodeId nodeId, Token containerToken) NodeId nodeId) throws YarnException, IOException {
throws YarnException, IOException {
NMCommunicator nmCommunicator = null; ContainerManagementProtocolProxyData proxy = null;
try { try {
nmCommunicator = new NMCommunicator(containerId, nodeId, containerToken); proxy = cmProxy.getProxy(nodeId.toString(), containerId);
nmCommunicator.init(getConfig()); ContainerStatus containerStatus =
nmCommunicator.start(); proxy.getContainerManagementProtocol().getContainerStatus(
ContainerStatus containerStatus = nmCommunicator.getContainerStatus(); GetContainerStatusRequest.newInstance(containerId)).getStatus();
return containerStatus; return containerStatus;
} finally { } finally {
if (nmCommunicator != null) { if (proxy != null) {
nmCommunicator.stop(); cmProxy.mayBeCloseProxy(proxy);
} }
} }
} }
protected synchronized StartedContainer addStartedContainer( private void stopContainerInternal(ContainerId containerId, NodeId nodeId)
Container container) throws YarnException, IOException { throws IOException, YarnException {
if (startedContainers.containsKey(container.getId())) { ContainerManagementProtocolProxyData proxy = null;
throw RPCUtil.getRemoteException("Container " + container.getId() + try {
" is already started"); proxy = cmProxy.getProxy(nodeId.toString(), containerId);
proxy.getContainerManagementProtocol().stopContainer(
StopContainerRequest.newInstance(containerId));
} finally {
if (proxy != null) {
cmProxy.mayBeCloseProxy(proxy);
}
} }
}
protected synchronized StartedContainer createStartedContainer(
Container container) throws YarnException, IOException {
StartedContainer startedContainer = new StartedContainer(container.getId(), StartedContainer startedContainer = new StartedContainer(container.getId(),
container.getNodeId(), container.getContainerToken()); container.getNodeId(), container.getContainerToken());
startedContainers.put(startedContainer.getContainerId(), startedContainer);
return startedContainer; return startedContainer;
} }
protected synchronized void removeStartedContainer(ContainerId containerId) { protected synchronized void
startedContainers.remove(containerId); removeStartedContainer(StartedContainer container) {
startedContainers.remove(container.containerId);
} }
protected synchronized StartedContainer getStartedContainer( protected synchronized StartedContainer getStartedContainer(
@ -405,5 +289,4 @@ public class NMClientImpl extends NMClient {
public AtomicBoolean getCleanupRunningContainers() { public AtomicBoolean getCleanupRunningContainers() {
return cleanupRunningContainers; return cleanupRunningContainers;
} }
} }

View File

@ -232,10 +232,10 @@ public class TestNMClientAsync {
actualStartSuccessArray.set(containerId.getId(), 1); actualStartSuccessArray.set(containerId.getId(), 1);
// move on to the following success tests // move on to the following success tests
asyncClient.getContainerStatusAsync(containerId, nodeId, containerToken); asyncClient.getContainerStatusAsync(containerId, nodeId);
} else { } else {
// move on to the following failure tests // move on to the following failure tests
asyncClient.stopContainerAsync(containerId, nodeId, containerToken); asyncClient.stopContainerAsync(containerId, nodeId);
} }
// Shouldn't crash the test thread // Shouldn't crash the test thread
@ -253,7 +253,7 @@ public class TestNMClientAsync {
actualQuerySuccess.addAndGet(1); actualQuerySuccess.addAndGet(1);
actualQuerySuccessArray.set(containerId.getId(), 1); actualQuerySuccessArray.set(containerId.getId(), 1);
// move on to the following success tests // move on to the following success tests
asyncClient.stopContainerAsync(containerId, nodeId, containerToken); asyncClient.stopContainerAsync(containerId, nodeId);
// Shouldn't crash the test thread // Shouldn't crash the test thread
throw new RuntimeException("Ignorable Exception"); throw new RuntimeException("Ignorable Exception");
@ -290,7 +290,7 @@ public class TestNMClientAsync {
actualStartFailure.addAndGet(1); actualStartFailure.addAndGet(1);
actualStartFailureArray.set(containerId.getId() - expectedSuccess, 1); actualStartFailureArray.set(containerId.getId() - expectedSuccess, 1);
// move on to the following failure tests // move on to the following failure tests
asyncClient.getContainerStatusAsync(containerId, nodeId, containerToken); asyncClient.getContainerStatusAsync(containerId, nodeId);
// Shouldn't crash the test thread // Shouldn't crash the test thread
throw new RuntimeException("Ignorable Exception"); throw new RuntimeException("Ignorable Exception");
@ -383,33 +383,30 @@ public class TestNMClientAsync {
when(client.startContainer(any(Container.class), when(client.startContainer(any(Container.class),
any(ContainerLaunchContext.class))).thenReturn( any(ContainerLaunchContext.class))).thenReturn(
Collections.<String, ByteBuffer>emptyMap()); Collections.<String, ByteBuffer>emptyMap());
when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class), when(client.getContainerStatus(any(ContainerId.class),
any(Token.class))).thenReturn( any(NodeId.class))).thenReturn(
recordFactory.newRecordInstance(ContainerStatus.class)); recordFactory.newRecordInstance(ContainerStatus.class));
doNothing().when(client).stopContainer(any(ContainerId.class), doNothing().when(client).stopContainer(any(ContainerId.class),
any(NodeId.class), any(Token.class)); any(NodeId.class));
break; break;
case 1: case 1:
doThrow(RPCUtil.getRemoteException("Start Exception")).when(client) doThrow(RPCUtil.getRemoteException("Start Exception")).when(client)
.startContainer(any(Container.class), .startContainer(any(Container.class),
any(ContainerLaunchContext.class)); any(ContainerLaunchContext.class));
doThrow(RPCUtil.getRemoteException("Query Exception")).when(client) doThrow(RPCUtil.getRemoteException("Query Exception")).when(client)
.getContainerStatus(any(ContainerId.class), any(NodeId.class), .getContainerStatus(any(ContainerId.class), any(NodeId.class));
any(Token.class));
doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client) doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
.stopContainer(any(ContainerId.class), any(NodeId.class), .stopContainer(any(ContainerId.class), any(NodeId.class));
any(Token.class));
break; break;
case 2: case 2:
when(client.startContainer(any(Container.class), when(client.startContainer(any(Container.class),
any(ContainerLaunchContext.class))).thenReturn( any(ContainerLaunchContext.class))).thenReturn(
Collections.<String, ByteBuffer>emptyMap()); Collections.<String, ByteBuffer>emptyMap());
when(client.getContainerStatus(any(ContainerId.class), any(NodeId.class), when(client.getContainerStatus(any(ContainerId.class),
any(Token.class))).thenReturn( any(NodeId.class))).thenReturn(
recordFactory.newRecordInstance(ContainerStatus.class)); recordFactory.newRecordInstance(ContainerStatus.class));
doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client) doThrow(RPCUtil.getRemoteException("Stop Exception")).when(client)
.stopContainer(any(ContainerId.class), any(NodeId.class), .stopContainer(any(ContainerId.class), any(NodeId.class));
any(Token.class));
} }
return client; return client;
} }
@ -437,8 +434,7 @@ public class TestNMClientAsync {
t.start(); t.start();
barrierA.await(); barrierA.await();
asyncClient.stopContainerAsync(container.getId(), container.getNodeId(), asyncClient.stopContainerAsync(container.getId(), container.getNodeId());
container.getContainerToken());
barrierC.await(); barrierC.await();
Assert.assertFalse("Starting and stopping should be out of order", Assert.assertFalse("Starting and stopping should be out of order",

View File

@ -29,6 +29,7 @@ import java.nio.ByteBuffer;
import java.util.List; import java.util.List;
import java.util.Set; import java.util.Set;
import java.util.TreeSet; import java.util.TreeSet;
import java.util.concurrent.ConcurrentHashMap;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.io.DataOutputBuffer; import org.apache.hadoop.io.DataOutputBuffer;
@ -46,10 +47,12 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.FinalApplicationStatus; import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeReport; import org.apache.hadoop.yarn.api.records.NodeReport;
import org.apache.hadoop.yarn.api.records.Priority; import org.apache.hadoop.yarn.api.records.Priority;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.YarnApplicationState; import org.apache.hadoop.yarn.api.records.YarnApplicationState;
import org.apache.hadoop.yarn.client.api.AMRMClient; import org.apache.hadoop.yarn.client.api.AMRMClient;
import org.apache.hadoop.yarn.client.api.NMClient; import org.apache.hadoop.yarn.client.api.NMClient;
@ -75,7 +78,8 @@ public class TestNMClient {
List<NodeReport> nodeReports = null; List<NodeReport> nodeReports = null;
ApplicationAttemptId attemptId = null; ApplicationAttemptId attemptId = null;
int nodeCount = 3; int nodeCount = 3;
ConcurrentHashMap<String, Token> nmTokens;
@Before @Before
public void setup() throws YarnException, IOException { public void setup() throws YarnException, IOException {
// start minicluster // start minicluster
@ -140,6 +144,7 @@ public class TestNMClient {
if (iterationsLeft == 0) { if (iterationsLeft == 0) {
fail("Application hasn't bee started"); fail("Application hasn't bee started");
} }
nmTokens = new ConcurrentHashMap<String, Token>();
// start am rm client // start am rm client
rmClient = rmClient =
@ -151,7 +156,7 @@ public class TestNMClient {
assertEquals(STATE.STARTED, rmClient.getServiceState()); assertEquals(STATE.STARTED, rmClient.getServiceState());
// start am nm client // start am nm client
nmClient = (NMClientImpl) NMClient.createNMClient(); nmClient = (NMClientImpl) NMClient.createNMClient(nmTokens);
nmClient.init(conf); nmClient.init(conf);
nmClient.start(); nmClient.start();
assertNotNull(nmClient); assertNotNull(nmClient);
@ -194,14 +199,13 @@ public class TestNMClient {
assertEquals(0, nmClient.startedContainers.size()); assertEquals(0, nmClient.startedContainers.size());
} }
@Test (timeout = 60000) @Test (timeout = 200000)
public void testNMClient() public void testNMClient()
throws YarnException, IOException { throws YarnException, IOException {
rmClient.registerApplicationMaster("Host", 10000, ""); rmClient.registerApplicationMaster("Host", 10000, "");
testContainerManagement(nmClient, allocateContainers(rmClient, 5)); testContainerManagement(nmClient, allocateContainers(rmClient, 5));
rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED, rmClient.unregisterApplicationMaster(FinalApplicationStatus.SUCCEEDED,
null, null); null, null);
// stop the running containers on close // stop the running containers on close
@ -243,6 +247,11 @@ public class TestNMClient {
for(Container container : allocResponse.getAllocatedContainers()) { for(Container container : allocResponse.getAllocatedContainers()) {
containers.add(container); containers.add(container);
} }
if (!allocResponse.getNMTokens().isEmpty()) {
for (NMToken token : allocResponse.getNMTokens()) {
nmTokens.put(token.getNodeId().toString(), token.getToken());
}
}
if(allocatedContainerCount < containersRequestedAny) { if(allocatedContainerCount < containersRequestedAny) {
// sleep to let NM's heartbeat to RM and trigger allocations // sleep to let NM's heartbeat to RM and trigger allocations
sleep(1000); sleep(1000);
@ -261,8 +270,7 @@ public class TestNMClient {
// getContainerStatus shouldn't be called before startContainer, // getContainerStatus shouldn't be called before startContainer,
// otherwise, NodeManager cannot find the container // otherwise, NodeManager cannot find the container
try { try {
nmClient.getContainerStatus(container.getId(), container.getNodeId(), nmClient.getContainerStatus(container.getId(), container.getNodeId());
container.getContainerToken());
fail("Exception is expected"); fail("Exception is expected");
} catch (YarnException e) { } catch (YarnException e) {
assertTrue("The thrown exception is not expected", assertTrue("The thrown exception is not expected",
@ -272,12 +280,11 @@ public class TestNMClient {
// stopContainer shouldn't be called before startContainer, // stopContainer shouldn't be called before startContainer,
// otherwise, an exception will be thrown // otherwise, an exception will be thrown
try { try {
nmClient.stopContainer(container.getId(), container.getNodeId(), nmClient.stopContainer(container.getId(), container.getNodeId());
container.getContainerToken());
fail("Exception is expected"); fail("Exception is expected");
} catch (YarnException e) { } catch (YarnException e) {
if (!e.getMessage() if (!e.getMessage()
.contains("is either not started yet or already stopped")) { .contains("is not handled by this NodeManager")) {
throw (AssertionError) throw (AssertionError)
(new AssertionError("Exception is not expected: " + e).initCause( (new AssertionError("Exception is not expected: " + e).initCause(
e)); e));
@ -306,8 +313,7 @@ public class TestNMClient {
-1000); -1000);
try { try {
nmClient.stopContainer(container.getId(), container.getNodeId(), nmClient.stopContainer(container.getId(), container.getNodeId());
container.getContainerToken());
} catch (YarnException e) { } catch (YarnException e) {
throw (AssertionError) throw (AssertionError)
(new AssertionError("Exception is not expected: " + e) (new AssertionError("Exception is not expected: " + e)
@ -335,8 +341,7 @@ public class TestNMClient {
while (true) { while (true) {
try { try {
ContainerStatus status = nmClient.getContainerStatus( ContainerStatus status = nmClient.getContainerStatus(
container.getId(), container.getNodeId(), container.getId(), container.getNodeId());
container.getContainerToken());
// NodeManager may still need some time to get the stable // NodeManager may still need some time to get the stable
// container status // container status
if (status.getState() == state) { if (status.getState() == state) {

View File

@ -55,7 +55,7 @@ public class ContainerManagerSecurityInfo extends SecurityInfo {
@Override @Override
public Class<? extends TokenSelector<? extends TokenIdentifier>> public Class<? extends TokenSelector<? extends TokenIdentifier>>
value() { value() {
return ContainerTokenSelector.class; return NMTokenSelector.class;
} }
}; };

View File

@ -21,21 +21,17 @@ package org.apache.hadoop.yarn.security;
import java.io.DataInput; import java.io.DataInput;
import java.io.DataOutput; import java.io.DataOutput;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Public; import org.apache.hadoop.classification.InterfaceAudience.Public;
import org.apache.hadoop.classification.InterfaceStability.Evolving; import org.apache.hadoop.classification.InterfaceStability.Evolving;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Token;
@Public @Public
@Evolving @Evolving
@ -48,14 +44,14 @@ public class NMTokenIdentifier extends TokenIdentifier {
private ApplicationAttemptId appAttemptId; private ApplicationAttemptId appAttemptId;
private NodeId nodeId; private NodeId nodeId;
private String appSubmitter; private String appSubmitter;
private int masterKeyId; private int keyId;
public NMTokenIdentifier(ApplicationAttemptId appAttemptId, NodeId nodeId, public NMTokenIdentifier(ApplicationAttemptId appAttemptId, NodeId nodeId,
String applicationSubmitter, int masterKeyId) { String applicationSubmitter, int masterKeyId) {
this.appAttemptId = appAttemptId; this.appAttemptId = appAttemptId;
this.nodeId = nodeId; this.nodeId = nodeId;
this.appSubmitter = applicationSubmitter; this.appSubmitter = applicationSubmitter;
this.masterKeyId = masterKeyId; this.keyId = masterKeyId;
} }
/** /**
@ -76,8 +72,8 @@ public class NMTokenIdentifier extends TokenIdentifier {
return appSubmitter; return appSubmitter;
} }
public int getMastKeyId() { public int getKeyId() {
return masterKeyId; return keyId;
} }
@Override @Override
@ -89,7 +85,7 @@ public class NMTokenIdentifier extends TokenIdentifier {
out.writeInt(appAttemptId.getAttemptId()); out.writeInt(appAttemptId.getAttemptId());
out.writeUTF(this.nodeId.toString()); out.writeUTF(this.nodeId.toString());
out.writeUTF(this.appSubmitter); out.writeUTF(this.appSubmitter);
out.writeInt(this.masterKeyId); out.writeInt(this.keyId);
} }
@Override @Override
@ -101,7 +97,7 @@ public class NMTokenIdentifier extends TokenIdentifier {
String[] hostAddr = in.readUTF().split(":"); String[] hostAddr = in.readUTF().split(":");
nodeId = NodeId.newInstance(hostAddr[0], Integer.parseInt(hostAddr[1])); nodeId = NodeId.newInstance(hostAddr[0], Integer.parseInt(hostAddr[1]));
appSubmitter = in.readUTF(); appSubmitter = in.readUTF();
masterKeyId = in.readInt(); keyId = in.readInt();
} }
@Override @Override

View File

@ -0,0 +1,56 @@
/**
* 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.security;
import java.util.Collection;
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.security.token.TokenSelector;
public class NMTokenSelector implements
TokenSelector<NMTokenIdentifier> {
private static final Log LOG = LogFactory
.getLog(NMTokenSelector.class);
@SuppressWarnings("unchecked")
@Override
public Token<NMTokenIdentifier> selectToken(Text service,
Collection<Token<? extends TokenIdentifier>> tokens) {
if (service == null) {
return null;
}
for (Token<? extends TokenIdentifier> token : tokens) {
if (LOG.isDebugEnabled()) {
LOG.info("Looking for service: " + service + ". Current token is "
+ token);
}
if (NMTokenIdentifier.KIND.equals(token.getKind()) &&
service.equals(token.getService())) {
return (Token<NMTokenIdentifier>) token;
}
}
return null;
}
}

View File

@ -15,3 +15,4 @@ org.apache.hadoop.yarn.security.ContainerTokenIdentifier
org.apache.hadoop.yarn.security.AMRMTokenIdentifier org.apache.hadoop.yarn.security.AMRMTokenIdentifier
org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier
org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier
org.apache.hadoop.yarn.security.NMTokenIdentifier

View File

@ -659,7 +659,25 @@
<name>yarn.client.nodemanager-client-async.thread-pool-max-size</name> <name>yarn.client.nodemanager-client-async.thread-pool-max-size</name>
<value>500</value> <value>500</value>
</property> </property>
<property>
<description>
Maximum number of proxy connections for node manager. It should always be
more than 1. NMClient and MRAppMaster will use this to cache connection
with node manager. There will be at max one connection per node manager.
Ex. configuring it to a value of 5 will make sure that client will at
max have 5 connections cached with 5 different node managers. These
connections will be timed out if idle for more than system wide idle
timeout period. The token if used for authentication then it will be used
only at connection creation time. If new token is received then earlier
connection should be closed in order to use newer token. This and
(yarn.client.nodemanager-client-async.thread-pool-max-size) are related
and should be sync (no need for them to be equal).
</description>
<name>yarn.client.max-nodemanagers-proxies</name>
<value>500</value>
</property>
<!--Map Reduce configuration--> <!--Map Reduce configuration-->
<property> <property>
<name>yarn.nodemanager.aux-services.mapreduce.shuffle.class</name> <name>yarn.nodemanager.aux-services.mapreduce.shuffle.class</name>

View File

@ -114,4 +114,38 @@ public class BaseNMTokenSecretManager extends
public NMTokenIdentifier createIdentifier() { public NMTokenIdentifier createIdentifier() {
return new NMTokenIdentifier(); return new NMTokenIdentifier();
} }
/**
* Helper function for creating NMTokens.
*/
public Token createNMToken(ApplicationAttemptId applicationAttemptId,
NodeId nodeId, String applicationSubmitter) {
byte[] password;
NMTokenIdentifier identifier;
this.readLock.lock();
try {
identifier =
new NMTokenIdentifier(applicationAttemptId, nodeId,
applicationSubmitter, this.currentMasterKey.getMasterKey()
.getKeyId());
password = this.createPassword(identifier);
} finally {
this.readLock.unlock();
}
return newInstance(password, identifier);
}
public static Token newInstance(byte[] password,
NMTokenIdentifier identifier) {
NodeId nodeId = identifier.getNodeId();
// RPC layer client expects ip:port as service for tokens
InetSocketAddress addr =
NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
Token nmToken =
Token.newInstance(identifier.getBytes(),
NMTokenIdentifier.KIND.toString(), password, SecurityUtil
.buildTokenService(addr).toString());
return nmToken;
}
} }

View File

@ -27,6 +27,7 @@ import java.util.Comparator;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil; import org.apache.hadoop.security.SecurityUtil;
@ -63,6 +64,8 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import com.google.common.annotations.VisibleForTesting;
/** /**
* Builder utilities to construct various objects. * Builder utilities to construct various objects.
* *
@ -152,7 +155,8 @@ public class BuilderUtils {
int port, String user, Resource r, long expiryTime, int masterKeyId, int port, String user, Resource r, long expiryTime, int masterKeyId,
byte[] password, long rmIdentifier) throws IOException { byte[] password, long rmIdentifier) throws IOException {
ContainerTokenIdentifier identifier = ContainerTokenIdentifier identifier =
new ContainerTokenIdentifier(cId, host, user, r, expiryTime, new ContainerTokenIdentifier(cId, host + ":" + port, user, r,
expiryTime,
masterKeyId, rmIdentifier); masterKeyId, rmIdentifier);
return newContainerToken(BuilderUtils.newNodeId(host, port), password, return newContainerToken(BuilderUtils.newNodeId(host, port), password,
identifier); identifier);
@ -228,6 +232,8 @@ public class BuilderUtils {
return newToken(Token.class, identifier, kind, password, service); return newToken(Token.class, identifier, kind, password, service);
} }
@Private
@VisibleForTesting
public static Token newContainerToken(NodeId nodeId, public static Token newContainerToken(NodeId nodeId,
byte[] password, ContainerTokenIdentifier tokenIdentifier) { byte[] password, ContainerTokenIdentifier tokenIdentifier) {
// RPC layer client expects ip:port as service for tokens // RPC layer client expects ip:port as service for tokens

View File

@ -435,7 +435,7 @@ public class NodeManager extends CompositeService
} }
@VisibleForTesting @VisibleForTesting
Context getNMContext() { public Context getNMContext() {
return this.context; return this.context;
} }

View File

@ -23,6 +23,7 @@ import static org.apache.hadoop.service.Service.STATE.STARTED;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Map; import java.util.Map;
import java.util.Set; import java.util.Set;
import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicBoolean;
@ -38,6 +39,7 @@ import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.authorize.PolicyProvider; import org.apache.hadoop.security.authorize.PolicyProvider;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.security.token.Token; import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.security.token.TokenIdentifier; import org.apache.hadoop.security.token.TokenIdentifier;
import org.apache.hadoop.service.CompositeService; import org.apache.hadoop.service.CompositeService;
@ -65,6 +67,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.RPCUtil; import org.apache.hadoop.yarn.ipc.RPCUtil;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedContainersEvent;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
@ -234,7 +237,7 @@ public class ContainerManagerImpl extends CompositeService implements
server = server =
rpc.getServer(ContainerManagementProtocol.class, this, initialAddress, conf, rpc.getServer(ContainerManagementProtocol.class, this, initialAddress, conf,
this.context.getContainerTokenSecretManager(), this.context.getNMTokenSecretManager(),
conf.getInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT, conf.getInt(YarnConfiguration.NM_CONTAINER_MGR_THREAD_COUNT,
YarnConfiguration.DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT)); YarnConfiguration.DEFAULT_NM_CONTAINER_MGR_THREAD_COUNT));
@ -253,6 +256,8 @@ public class ContainerManagerImpl extends CompositeService implements
NodeId nodeId = NodeId.newInstance(connectAddress.getHostName(), NodeId nodeId = NodeId.newInstance(connectAddress.getHostName(),
connectAddress.getPort()); connectAddress.getPort());
((NodeManager.NMContext)context).setNodeId(nodeId); ((NodeManager.NMContext)context).setNodeId(nodeId);
this.context.getNMTokenSecretManager().setNodeId(nodeId);
this.context.getContainerTokenSecretManager().setNodeId(nodeId);
LOG.info("ContainerManager started at " + connectAddress); LOG.info("ContainerManager started at " + connectAddress);
super.serviceStart(); super.serviceStart();
} }
@ -274,7 +279,7 @@ public class ContainerManagerImpl extends CompositeService implements
} }
// Get the remoteUGI corresponding to the api call. // Get the remoteUGI corresponding to the api call.
private UserGroupInformation getRemoteUgi() protected UserGroupInformation getRemoteUgi()
throws YarnException { throws YarnException {
UserGroupInformation remoteUgi; UserGroupInformation remoteUgi;
try { try {
@ -291,91 +296,67 @@ public class ContainerManagerImpl extends CompositeService implements
// Obtain the needed ContainerTokenIdentifier from the remote-UGI. RPC layer // Obtain the needed ContainerTokenIdentifier from the remote-UGI. RPC layer
// currently sets only the required id, but iterate through anyways just to // currently sets only the required id, but iterate through anyways just to
// be sure. // be sure.
private ContainerTokenIdentifier selectContainerTokenIdentifier( @Private
@VisibleForTesting
protected NMTokenIdentifier selectNMTokenIdentifier(
UserGroupInformation remoteUgi) { UserGroupInformation remoteUgi) {
Set<TokenIdentifier> tokenIdentifiers = remoteUgi.getTokenIdentifiers(); Set<TokenIdentifier> tokenIdentifiers = remoteUgi.getTokenIdentifiers();
ContainerTokenIdentifier resultId = null; NMTokenIdentifier resultId = null;
for (TokenIdentifier id : tokenIdentifiers) { for (TokenIdentifier id : tokenIdentifiers) {
if (id instanceof ContainerTokenIdentifier) { if (id instanceof NMTokenIdentifier) {
resultId = (ContainerTokenIdentifier) id; resultId = (NMTokenIdentifier) id;
break; break;
} }
} }
return resultId; return resultId;
} }
@Private
@VisibleForTesting
protected ContainerTokenIdentifier getContainerTokenIdentifier(
UserGroupInformation remoteUgi,
ContainerTokenIdentifier containerTokenIdentifier)
throws YarnException {
if (UserGroupInformation.isSecurityEnabled()) {
if (LOG.isDebugEnabled()) {
LOG.debug("Number of TokenIdentifiers in the UGI from RPC: "
+ remoteUgi.getTokenIdentifiers().size());
}
// Get the tokenId from the remote user ugi
return selectContainerTokenIdentifier(remoteUgi);
} else {
return containerTokenIdentifier;
}
}
/** /**
* Authorize the request. * @param containerTokenIdentifier
* * of the container to be started
* @param containerIDStr * @param ugi
* of the container
* @param launchContext
* passed if verifying the startContainer, null otherwise.
* @param remoteUgi
* ugi corresponding to the remote end making the api-call * ugi corresponding to the remote end making the api-call
* @throws YarnException * @throws YarnException
*/ */
@Private @Private
@VisibleForTesting @VisibleForTesting
protected void authorizeRequest(String containerIDStr, protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
ContainerLaunchContext launchContext, ContainerTokenIdentifier containerTokenIdentifier,
UserGroupInformation remoteUgi, ContainerTokenIdentifier tokenId) UserGroupInformation ugi) throws YarnException {
throws YarnException {
ContainerId containerId = containerTokenIdentifier.getContainerID();
String containerIDStr = containerId.toString();
boolean unauthorized = false; boolean unauthorized = false;
StringBuilder messageBuilder = StringBuilder messageBuilder =
new StringBuilder("Unauthorized request to start container. "); new StringBuilder("Unauthorized request to start container. ");
if (!nmTokenIdentifier.getApplicationAttemptId().equals(
if (!remoteUgi.getUserName().equals(containerIDStr)) { containerId.getApplicationAttemptId())) {
unauthorized = true; unauthorized = true;
messageBuilder.append("\nExpected containerId: " messageBuilder.append("\nNMToken for application attempt : ")
+ remoteUgi.getUserName() + " Found: " + containerIDStr); .append(nmTokenIdentifier.getApplicationAttemptId())
} else if (launchContext != null) { .append(" was used for starting container with container token")
// Verify other things also for startContainer() request. .append(" issued for application attempt : ")
.append(containerId.getApplicationAttemptId());
} else if (!ugi.getUserName().equals(
if (tokenId == null) { nmTokenIdentifier.getApplicationAttemptId().toString())) {
unauthorized = true; unauthorized = true;
messageBuilder messageBuilder.append("\nExpected applicationAttemptId: ")
.append("\nNo ContainerToken found for " + containerIDStr); .append(ugi.getUserName()).append(" Found: ")
} else { .append(nmTokenIdentifier.getApplicationAttemptId().toString());
} else if (!this.context.getContainerTokenSecretManager()
// Is the container being relaunched? Or RPC layer let startCall with .isValidStartContainerRequest(containerId)) {
// tokens generated off old-secret through? // Is the container being relaunched? Or RPC layer let startCall with
if (!this.context.getContainerTokenSecretManager() // tokens generated off old-secret through?
.isValidStartContainerRequest(tokenId.getContainerID())) { unauthorized = true;
unauthorized = true; messageBuilder.append("\n Attempt to relaunch the same ")
messageBuilder.append("\n Attempt to relaunch the same " .append("container with id ").append(containerIDStr).append(".");
+ "container with id " + containerIDStr + "."); } else if (containerTokenIdentifier.getExpiryTimeStamp() < System
} .currentTimeMillis()) {
// Ensure the token is not expired.
// Ensure the token is not expired. unauthorized = true;
// Token expiry is not checked for stopContainer/getContainerStatus messageBuilder.append("\nThis token is expired. current time is ")
if (tokenId.getExpiryTimeStamp() < System.currentTimeMillis()) { .append(System.currentTimeMillis()).append(" found ")
unauthorized = true; .append(containerTokenIdentifier.getExpiryTimeStamp());
messageBuilder.append("\nThis token is expired. current time is "
+ System.currentTimeMillis() + " found "
+ tokenId.getExpiryTimeStamp());
}
}
} }
if (unauthorized) { if (unauthorized) {
@ -384,7 +365,7 @@ public class ContainerManagerImpl extends CompositeService implements
throw RPCUtil.getRemoteException(msg); throw RPCUtil.getRemoteException(msg);
} }
} }
/** /**
* Start a container on this NodeManager. * Start a container on this NodeManager.
*/ */
@ -395,44 +376,133 @@ public class ContainerManagerImpl extends CompositeService implements
if (blockNewContainerRequests.get()) { if (blockNewContainerRequests.get()) {
throw new NMNotYetReadyException( throw new NMNotYetReadyException(
"Rejecting new containers as NodeManager has not" + "Rejecting new containers as NodeManager has not"
" yet connected with ResourceManager"); + " yet connected with ResourceManager");
} }
/*
* 1) It should save the NMToken into NMTokenSecretManager. This is done
* here instead of RPC layer because at the time of opening/authenticating
* the connection it doesn't know what all RPC calls user will make on it.
* Also new NMToken is issued only at startContainer (once it gets renewed).
*
* 2) It should validate containerToken. Need to check below things. a) It
* is signed by correct master key (part of retrieve password). b) It
* belongs to correct Node Manager (part of retrieve password). c) It has
* correct RMIdentifier. d) It is not expired.
*/
// update NMToken
UserGroupInformation remoteUgi = getRemoteUgi();
NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
// Validate containerToken
ContainerTokenIdentifier containerTokenIdentifier =
verifyAndGetContainerTokenIdentifier(request.getContainerToken());
authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier,
remoteUgi);
if (containerTokenIdentifier.getRMIdentifer() != nodeStatusUpdater
.getRMIdentifier()) {
// Is the container coming from unknown RM
StringBuilder sb = new StringBuilder("\nContainer ");
sb.append(containerTokenIdentifier.getContainerID().toString())
.append(" rejected as it is allocated by a previous RM");
throw new InvalidContainerException(sb.toString());
}
updateNMTokenIdentifier(nmTokenIdentifier);
ContainerId containerId = containerTokenIdentifier.getContainerID();
String containerIdStr = containerId.toString();
String user = containerTokenIdentifier.getApplicationSubmitter();
LOG.info("Start request for " + containerIdStr + " by user " + user);
ContainerLaunchContext launchContext = request.getContainerLaunchContext(); ContainerLaunchContext launchContext = request.getContainerLaunchContext();
org.apache.hadoop.yarn.api.records.Token token = request.getContainerToken();
ContainerTokenIdentifier tokenIdentifier = null; Credentials credentials = parseCredentials(launchContext);
Container container =
new ContainerImpl(getConfig(), this.dispatcher, launchContext,
credentials, metrics, containerTokenIdentifier);
ApplicationId applicationID =
containerId.getApplicationAttemptId().getApplicationId();
if (context.getContainers().putIfAbsent(containerId, container) != null) {
NMAuditLogger.logFailure(user, AuditConstants.START_CONTAINER,
"ContainerManagerImpl", "Container already running on this node!",
applicationID, containerId);
throw RPCUtil.getRemoteException("Container " + containerIdStr
+ " already is running on this node!!");
}
// Create the application
Application application =
new ApplicationImpl(dispatcher, this.aclsManager, user, applicationID,
credentials, context);
if (null == context.getApplications().putIfAbsent(applicationID,
application)) {
LOG.info("Creating a new application reference for app " + applicationID);
dispatcher.getEventHandler().handle(
new ApplicationInitEvent(applicationID, container.getLaunchContext()
.getApplicationACLs()));
}
dispatcher.getEventHandler().handle(
new ApplicationContainerInitEvent(container));
this.context.getContainerTokenSecretManager().startContainerSuccessful(
containerTokenIdentifier);
NMAuditLogger.logSuccess(user, AuditConstants.START_CONTAINER,
"ContainerManageImpl", applicationID, containerId);
StartContainerResponse response =
recordFactory.newRecordInstance(StartContainerResponse.class);
response.setAllServicesMetaData(auxiliaryServices.getMetaData());
// TODO launchedContainer misplaced -> doesn't necessarily mean a container
// launch. A finished Application will not launch containers.
metrics.launchedContainer();
metrics.allocateContainer(containerTokenIdentifier.getResource());
return response;
}
protected ContainerTokenIdentifier verifyAndGetContainerTokenIdentifier(
org.apache.hadoop.yarn.api.records.Token token) throws YarnException,
InvalidToken {
ContainerTokenIdentifier containerTokenIdentifier = null;
try { try {
tokenIdentifier = BuilderUtils.newContainerTokenIdentifier(token); containerTokenIdentifier =
BuilderUtils.newContainerTokenIdentifier(token);
} catch (IOException e) { } catch (IOException e) {
throw RPCUtil.getRemoteException(e); throw RPCUtil.getRemoteException(e);
} }
byte[] password =
UserGroupInformation remoteUgi = getRemoteUgi(); context.getContainerTokenSecretManager().retrievePassword(
ContainerTokenIdentifier tokenId = containerTokenIdentifier);
getContainerTokenIdentifier(remoteUgi, tokenIdentifier); byte[] tokenPass = token.getPassword().array();
if (password == null || tokenPass == null
ContainerId containerID = tokenId.getContainerID(); || !Arrays.equals(password, tokenPass)) {
String containerIDStr = containerID.toString(); throw new InvalidToken(
"Invalid container token used for starting container on : "
authorizeRequest(containerIDStr, launchContext, remoteUgi, tokenId); + context.getNodeId().toString());
// Is the container coming from unknown RM
if (tokenId.getRMIdentifer() != nodeStatusUpdater
.getRMIdentifier()) {
String msg = "\nContainer "+ containerIDStr
+ " rejected as it is allocated by a previous RM";
LOG.error(msg);
throw new InvalidContainerException(msg);
} }
return containerTokenIdentifier;
}
LOG.info("Start request for " + containerIDStr + " by user " @Private
+ tokenId.getApplicationSubmitter()); @VisibleForTesting
protected void updateNMTokenIdentifier(NMTokenIdentifier nmTokenIdentifier)
throws InvalidToken {
context.getNMTokenSecretManager().appAttemptStartContainer(
nmTokenIdentifier);
}
private Credentials parseCredentials(ContainerLaunchContext launchContext)
throws YarnException {
Credentials credentials = new Credentials();
// //////////// Parse credentials // //////////// Parse credentials
ByteBuffer tokens = launchContext.getTokens(); ByteBuffer tokens = launchContext.getTokens();
Credentials credentials = new Credentials();
if (tokens != null) { if (tokens != null) {
DataInputByteBuffer buf = new DataInputByteBuffer(); DataInputByteBuffer buf = new DataInputByteBuffer();
tokens.rewind(); tokens.rewind();
@ -440,8 +510,7 @@ public class ContainerManagerImpl extends CompositeService implements
try { try {
credentials.readTokenStorageStream(buf); credentials.readTokenStorageStream(buf);
if (LOG.isDebugEnabled()) { if (LOG.isDebugEnabled()) {
for (Token<? extends TokenIdentifier> tk : credentials for (Token<? extends TokenIdentifier> tk : credentials.getAllTokens()) {
.getAllTokens()) {
LOG.debug(tk.getService() + " = " + tk.toString()); LOG.debug(tk.getService() + " = " + tk.toString());
} }
} }
@ -450,53 +519,7 @@ public class ContainerManagerImpl extends CompositeService implements
} }
} }
// //////////// End of parsing credentials // //////////// End of parsing credentials
String user = tokenId.getApplicationSubmitter(); return credentials;
Container container =
new ContainerImpl(getConfig(), this.dispatcher, launchContext,
credentials, metrics, tokenId);
ApplicationId applicationID =
containerID.getApplicationAttemptId().getApplicationId();
if (context.getContainers().putIfAbsent(containerID, container) != null) {
NMAuditLogger.logFailure(user,
AuditConstants.START_CONTAINER, "ContainerManagerImpl",
"Container already running on this node!",
applicationID, containerID);
throw RPCUtil.getRemoteException("Container " + containerIDStr
+ " already is running on this node!!");
}
// Create the application
Application application =
new ApplicationImpl(dispatcher, this.aclsManager,
user, applicationID, credentials,
context);
if (null ==
context.getApplications().putIfAbsent(applicationID, application)) {
LOG.info("Creating a new application reference for app "
+ applicationID);
dispatcher.getEventHandler().handle(
new ApplicationInitEvent(applicationID, container
.getLaunchContext().getApplicationACLs()));
}
// TODO: Validate the request
dispatcher.getEventHandler().handle(
new ApplicationContainerInitEvent(container));
this.context.getContainerTokenSecretManager().startContainerSuccessful(
tokenId);
NMAuditLogger.logSuccess(user,
AuditConstants.START_CONTAINER, "ContainerManageImpl",
applicationID, containerID);
StartContainerResponse response =
StartContainerResponse.newInstance(auxiliaryServices.getMetaData());
// TODO launchedContainer misplaced -> doesn't necessarily mean a container
// launch. A finished Application will not launch containers.
metrics.launchedContainer();
metrics.allocateContainer(tokenId.getResource());
return response;
} }
/** /**
@ -509,34 +532,20 @@ public class ContainerManagerImpl extends CompositeService implements
ContainerId containerID = request.getContainerId(); ContainerId containerID = request.getContainerId();
String containerIDStr = containerID.toString(); String containerIDStr = containerID.toString();
// TODO: Only the container's owner can kill containers today.
UserGroupInformation remoteUgi = getRemoteUgi();
Container container = this.context.getContainers().get(containerID); Container container = this.context.getContainers().get(containerID);
LOG.info("Getting container-status for " + containerIDStr);
authorizeGetAndStopContainerRequest(containerID, container, true);
StopContainerResponse response = StopContainerResponse response =
recordFactory.newRecordInstance(StopContainerResponse.class); recordFactory.newRecordInstance(StopContainerResponse.class);
if (container == null) {
LOG.warn("Trying to stop unknown container " + containerID);
NMAuditLogger.logFailure("UnknownUser",
AuditConstants.STOP_CONTAINER, "ContainerManagerImpl",
"Trying to stop unknown container!",
containerID.getApplicationAttemptId().getApplicationId(),
containerID);
return response; // Return immediately.
}
authorizeRequest(containerIDStr, null, remoteUgi,
getContainerTokenIdentifier(remoteUgi, container.getContainerTokenIdentifier()));
dispatcher.getEventHandler().handle( dispatcher.getEventHandler().handle(
new ContainerKillEvent(containerID, new ContainerKillEvent(containerID,
"Container killed by the ApplicationMaster.")); "Container killed by the ApplicationMaster."));
NMAuditLogger.logSuccess(container.getUser(), NMAuditLogger.logSuccess(container.getUser(),
AuditConstants.STOP_CONTAINER, "ContainerManageImpl", AuditConstants.STOP_CONTAINER, "ContainerManageImpl", containerID
containerID.getApplicationAttemptId().getApplicationId(), .getApplicationAttemptId().getApplicationId(), containerID);
containerID);
// TODO: Move this code to appropriate place once kill_container is // TODO: Move this code to appropriate place once kill_container is
// implemented. // implemented.
@ -547,23 +556,14 @@ public class ContainerManagerImpl extends CompositeService implements
@Override @Override
public GetContainerStatusResponse getContainerStatus( public GetContainerStatusResponse getContainerStatus(
GetContainerStatusRequest request) throws YarnException, GetContainerStatusRequest request) throws YarnException, IOException {
IOException {
ContainerId containerID = request.getContainerId(); ContainerId containerID = request.getContainerId();
String containerIDStr = containerID.toString(); String containerIDStr = containerID.toString();
// TODO: Only the container's owner can get containers' status today.
UserGroupInformation remoteUgi = getRemoteUgi();
LOG.info("Getting container-status for " + containerIDStr);
Container container = this.context.getContainers().get(containerID); Container container = this.context.getContainers().get(containerID);
if (container == null) {
throw RPCUtil.getRemoteException("Container " + containerIDStr LOG.info("Getting container-status for " + containerIDStr);
+ " is not handled by this NodeManager"); authorizeGetAndStopContainerRequest(containerID, container, false);
}
authorizeRequest(containerIDStr, null, remoteUgi,
getContainerTokenIdentifier(remoteUgi, container.getContainerTokenIdentifier()));
ContainerStatus containerStatus = container.cloneAndGetContainerStatus(); ContainerStatus containerStatus = container.cloneAndGetContainerStatus();
LOG.info("Returning " + containerStatus); LOG.info("Returning " + containerStatus);
@ -573,6 +573,48 @@ public class ContainerManagerImpl extends CompositeService implements
return response; return response;
} }
@Private
@VisibleForTesting
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
Container container, boolean stopRequest) throws YarnException {
UserGroupInformation remoteUgi = getRemoteUgi();
NMTokenIdentifier identifier = selectNMTokenIdentifier(remoteUgi);
/*
* For get/stop container status; we need to verify that 1) User (NMToken)
* application attempt only has started container. 2) Requested containerId
* belongs to the same application attempt (NMToken) which was used. (Note:-
* This will prevent user in knowing another application's containers).
*/
if ((!identifier.getApplicationAttemptId().equals(
containerId.getApplicationAttemptId()))
|| (container != null && !identifier.getApplicationAttemptId().equals(
container.getContainerId().getApplicationAttemptId()))) {
if (stopRequest) {
LOG.warn(identifier.getApplicationAttemptId()
+ " attempted to stop non-application container : "
+ container.getContainerId().toString());
NMAuditLogger.logFailure("UnknownUser", AuditConstants.STOP_CONTAINER,
"ContainerManagerImpl", "Trying to stop unknown container!",
identifier.getApplicationAttemptId().getApplicationId(),
container.getContainerId());
} else {
LOG.warn(identifier.getApplicationAttemptId()
+ " attempted to get get status for non-application container : "
+ container.getContainerId().toString());
}
throw RPCUtil.getRemoteException("Container " + containerId.toString()
+ " is not started by this application attempt.");
}
if (container == null) {
throw RPCUtil.getRemoteException("Container " + containerId.toString()
+ " is not handled by this NodeManager");
}
}
class ContainerEventDispatcher implements EventHandler<ContainerEvent> { class ContainerEventDispatcher implements EventHandler<ContainerEvent> {
@Override @Override
public void handle(ContainerEvent event) { public void handle(ContainerEvent event) {
@ -643,9 +685,19 @@ public class ContainerManagerImpl extends CompositeService implements
this.blockNewContainerRequests.set(blockNewContainerRequests); this.blockNewContainerRequests.set(blockNewContainerRequests);
} }
@Private
@VisibleForTesting
public boolean getBlockNewContainerRequestsStatus() {
return this.blockNewContainerRequests.get();
}
@Override @Override
public void stateChanged(Service service) { public void stateChanged(Service service) {
// TODO Auto-generated method stub // TODO Auto-generated method stub
} }
public Context getContext() {
return this.context;
}
} }

View File

@ -18,26 +18,32 @@
package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security; package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.security;
import javax.crypto.SecretKey;
import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager;
public class LocalizerTokenSecretManager extends public class LocalizerTokenSecretManager extends
SecretManager<LocalizerTokenIdentifier> { SecretManager<LocalizerTokenIdentifier> {
private final SecretKey secretKey;
public LocalizerTokenSecretManager() {
this.secretKey = generateSecret();
}
@Override @Override
protected byte[] createPassword(LocalizerTokenIdentifier identifier) { protected byte[] createPassword(LocalizerTokenIdentifier identifier) {
return "testing".getBytes(); return createPassword(identifier.getBytes(), secretKey);
} }
@Override @Override
public byte[] retrievePassword(LocalizerTokenIdentifier identifier) public byte[] retrievePassword(LocalizerTokenIdentifier identifier)
throws org.apache.hadoop.security.token.SecretManager.InvalidToken { throws org.apache.hadoop.security.token.SecretManager.InvalidToken {
// TODO Auto-generated method stub return createPassword(identifier.getBytes(), secretKey);
return "testing".getBytes();
} }
@Override @Override
public LocalizerTokenIdentifier createIdentifier() { public LocalizerTokenIdentifier createIdentifier() {
// TODO Auto-generated method stub
return new LocalizerTokenIdentifier(); return new LocalizerTokenIdentifier();
} }

View File

@ -30,13 +30,12 @@ import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.SecretManager; import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager; import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.MasterKeyData; import org.apache.hadoop.yarn.server.security.MasterKeyData;
import com.google.common.annotations.VisibleForTesting;
/** /**
* The NM maintains only two master-keys. The current key that RM knows and the * The NM maintains only two master-keys. The current key that RM knows and the
* key from the previous rolling-interval. * key from the previous rolling-interval.
@ -51,6 +50,7 @@ public class NMContainerTokenSecretManager extends
private MasterKeyData previousMasterKey; private MasterKeyData previousMasterKey;
private final Map<ApplicationId, ConcurrentMap<ContainerId, MasterKeyData>> oldMasterKeys; private final Map<ApplicationId, ConcurrentMap<ContainerId, MasterKeyData>> oldMasterKeys;
private String nodeHostAddr;
public NMContainerTokenSecretManager(Configuration conf) { public NMContainerTokenSecretManager(Configuration conf) {
super(conf); super(conf);
@ -122,6 +122,15 @@ public class NMContainerTokenSecretManager extends
masterKeyToUse = this.oldMasterKeys.get(appId).get(containerId); masterKeyToUse = this.oldMasterKeys.get(appId).get(containerId);
} }
if (nodeHostAddr != null
&& !identifier.getNmHostAddress().equals(nodeHostAddr)) {
// Valid container token used for incorrect node.
throw new SecretManager.InvalidToken("Given Container "
+ identifier.getContainerID().toString()
+ " identifier is not valid for current Node manager. Expected : "
+ nodeHostAddr + " Found : " + identifier.getNmHostAddress());
}
if (masterKeyToUse != null) { if (masterKeyToUse != null) {
return retrievePasswordInternal(identifier, masterKeyToUse); return retrievePasswordInternal(identifier, masterKeyToUse);
} }
@ -186,4 +195,9 @@ public class NMContainerTokenSecretManager extends
public synchronized void appFinished(ApplicationId appId) { public synchronized void appFinished(ApplicationId appId) {
this.oldMasterKeys.remove(appId); this.oldMasterKeys.remove(appId);
} }
public synchronized void setNodeId(NodeId nodeId) {
nodeHostAddr = nodeId.toString();
LOG.info("Updating node address : " + nodeHostAddr);
}
} }

View File

@ -18,18 +18,24 @@
package org.apache.hadoop.yarn.server.nodemanager.security; package org.apache.hadoop.yarn.server.nodemanager.security;
import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List;
import java.util.Map; import java.util.Map;
import org.apache.commons.logging.Log; import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.security.NMTokenIdentifier; import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager; import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager;
import org.apache.hadoop.yarn.server.security.MasterKeyData; import org.apache.hadoop.yarn.server.security.MasterKeyData;
import com.google.common.annotations.VisibleForTesting;
public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager { public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
private static final Log LOG = LogFactory private static final Log LOG = LogFactory
@ -38,10 +44,15 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
private MasterKeyData previousMasterKey; private MasterKeyData previousMasterKey;
private final Map<ApplicationAttemptId, MasterKeyData> oldMasterKeys; private final Map<ApplicationAttemptId, MasterKeyData> oldMasterKeys;
private final Map<ApplicationId, List<ApplicationAttemptId>> appToAppAttemptMap;
private NodeId nodeId;
public NMTokenSecretManagerInNM() { public NMTokenSecretManagerInNM() {
this.oldMasterKeys = this.oldMasterKeys =
new HashMap<ApplicationAttemptId, MasterKeyData>(); new HashMap<ApplicationAttemptId, MasterKeyData>();
appToAppAttemptMap =
new HashMap<ApplicationId, List<ApplicationAttemptId>>();
} }
/** /**
@ -69,46 +80,117 @@ public class NMTokenSecretManagerInNM extends BaseNMTokenSecretManager {
} }
/** /**
* This method will be used to verify NMTokens generated by different * This method will be used to verify NMTokens generated by different master
* master keys. * keys.
*/ */
@Override @Override
public synchronized byte[] retrievePassword( public synchronized byte[] retrievePassword(NMTokenIdentifier identifier)
NMTokenIdentifier identifier) throws InvalidToken { throws InvalidToken {
int keyId = identifier.getMastKeyId(); int keyId = identifier.getKeyId();
ApplicationAttemptId appAttemptId = identifier.getApplicationAttemptId(); ApplicationAttemptId appAttemptId = identifier.getApplicationAttemptId();
/* /*
* MasterKey used for retrieving password will be as follows. * MasterKey used for retrieving password will be as follows. 1) By default
* 1) By default older saved master key will be used. * older saved master key will be used. 2) If identifier's master key id
* 2) If identifier's master key id matches that of previous master key * matches that of previous master key id then previous key will be used. 3)
* id then previous key will be used. * If identifier's master key id matches that of current master key id then
* 3) If identifier's master key id matches that of current master key * current key will be used.
* id then current key will be used.
*/ */
MasterKeyData oldMasterKey = oldMasterKeys.get(appAttemptId); MasterKeyData oldMasterKey = oldMasterKeys.get(appAttemptId);
MasterKeyData masterKeyToUse = oldMasterKey; MasterKeyData masterKeyToUse = oldMasterKey;
if (previousMasterKey != null if (previousMasterKey != null
&& keyId == previousMasterKey.getMasterKey().getKeyId()) { && keyId == previousMasterKey.getMasterKey().getKeyId()) {
masterKeyToUse = previousMasterKey; masterKeyToUse = previousMasterKey;
} else if ( keyId == currentMasterKey.getMasterKey().getKeyId()) { } else if (keyId == currentMasterKey.getMasterKey().getKeyId()) {
masterKeyToUse = currentMasterKey; masterKeyToUse = currentMasterKey;
} }
if (nodeId != null && !identifier.getNodeId().equals(nodeId)) {
throw new InvalidToken("Given NMToken for application : "
+ appAttemptId.toString() + " is not valid for current node manager."
+ "expected : " + nodeId.toString() + " found : "
+ identifier.getNodeId().toString());
}
if (masterKeyToUse != null) { if (masterKeyToUse != null) {
byte[] password = retrivePasswordInternal(identifier, masterKeyToUse); byte[] password = retrivePasswordInternal(identifier, masterKeyToUse);
if (masterKeyToUse.getMasterKey().getKeyId() != oldMasterKey LOG.debug("NMToken password retrieved successfully!!");
.getMasterKey().getKeyId()) {
oldMasterKeys.put(appAttemptId, masterKeyToUse);
}
return password; return password;
} }
throw new InvalidToken("Given NMToken for application : " throw new InvalidToken("Given NMToken for application : "
+ appAttemptId.toString() + " seems to have been generated illegally."); + appAttemptId.toString() + " seems to have been generated illegally.");
} }
public synchronized void appFinished(ApplicationId appId) {
List<ApplicationAttemptId> appAttemptList = appToAppAttemptMap.get(appId);
if (appAttemptList != null) {
LOG.debug("Removing application attempts NMToken keys for application "
+ appId);
for (ApplicationAttemptId appAttemptId : appAttemptList) {
this.oldMasterKeys.remove(appAttemptId);
}
appToAppAttemptMap.remove(appId);
} else {
LOG.error("No application Attempt for application : " + appId
+ " started on this NM.");
}
}
/**
* This will be called by startContainer. It will add the master key into
* the cache used for starting this container. This should be called before
* validating the startContainer request.
*/
public synchronized void appAttemptStartContainer(
NMTokenIdentifier identifier)
throws org.apache.hadoop.security.token.SecretManager.InvalidToken {
ApplicationAttemptId appAttemptId = identifier.getApplicationAttemptId();
if (!appToAppAttemptMap.containsKey(appAttemptId.getApplicationId())) {
// First application attempt for the given application
appToAppAttemptMap.put(appAttemptId.getApplicationId(),
new ArrayList<ApplicationAttemptId>());
}
MasterKeyData oldKey = oldMasterKeys.get(appAttemptId);
if (oldKey == null) {
// This is a new application attempt.
appToAppAttemptMap.get(appAttemptId.getApplicationId()).add(appAttemptId);
}
if (oldKey == null
|| oldKey.getMasterKey().getKeyId() != identifier.getKeyId()) {
// Update key only if it is modified.
LOG.debug("NMToken key updated for application attempt : "
+ identifier.getApplicationAttemptId().toString());
if (identifier.getKeyId() == currentMasterKey.getMasterKey()
.getKeyId()) {
oldMasterKeys.put(appAttemptId, currentMasterKey);
} else if (previousMasterKey != null
&& identifier.getKeyId() == previousMasterKey.getMasterKey()
.getKeyId()) {
oldMasterKeys.put(appAttemptId, previousMasterKey);
} else {
throw new InvalidToken(
"Older NMToken should not be used while starting the container.");
}
}
}
public synchronized void appFinished(ApplicationAttemptId appAttemptId) { public synchronized void setNodeId(NodeId nodeId) {
this.oldMasterKeys.remove(appAttemptId); LOG.debug("updating nodeId : " + nodeId);
this.nodeId = nodeId;
}
@Private
@VisibleForTesting
public synchronized boolean
isAppAttemptNMTokenKeyPresent(ApplicationAttemptId appAttemptId) {
return oldMasterKeys.containsKey(appAttemptId);
}
@Private
@VisibleForTesting
public synchronized NodeId getNodeId() {
return this.nodeId;
} }
} }

View File

@ -27,10 +27,12 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
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.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
@ -125,6 +127,19 @@ public class DummyContainerManager extends ContainerManagerImpl {
}; };
} }
@Override
protected UserGroupInformation getRemoteUgi() throws YarnException {
ApplicationId appId = ApplicationId.newInstance(0, 0);
ApplicationAttemptId appAttemptId =
ApplicationAttemptId.newInstance(appId, 1);
UserGroupInformation ugi =
UserGroupInformation.createRemoteUser(appAttemptId.toString());
ugi.addTokenIdentifier(new NMTokenIdentifier(appAttemptId, getContext()
.getNodeId(), "testuser", getContext().getNMTokenSecretManager().getCurrentKey()
.getKeyId()));
return ugi;
}
@Override @Override
@SuppressWarnings("unchecked") @SuppressWarnings("unchecked")
protected ContainersLauncher createContainersLauncher(Context context, protected ContainersLauncher createContainersLauncher(Context context,
@ -179,17 +194,16 @@ public class DummyContainerManager extends ContainerManagerImpl {
} }
@Override @Override
protected void authorizeRequest(String containerIDStr, protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
ContainerLaunchContext launchContext, ContainerTokenIdentifier containerTokenIdentifier,
UserGroupInformation remoteUgi, ContainerTokenIdentifier tokenId) UserGroupInformation ugi) throws YarnException {
throws YarnException { // do nothing
// do Nothing }
@Override
protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
Container container, boolean stopRequest) throws YarnException {
// do nothing
} }
@Override
protected ContainerTokenIdentifier
getContainerTokenIdentifier(UserGroupInformation remoteUgi,
ContainerTokenIdentifier containerTokenId) throws YarnException {
return containerTokenId;
}
} }

View File

@ -74,7 +74,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
} }
@Override @Override
public void testContainerSetup() throws IOException, InterruptedException, public void testContainerSetup() throws Exception, InterruptedException,
YarnException { YarnException {
// Don't run the test if the binary is not available. // Don't run the test if the binary is not available.
if (!shouldRunTest()) { if (!shouldRunTest()) {

View File

@ -41,6 +41,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.api.ResourceTracker; import org.apache.hadoop.yarn.server.api.ResourceTracker;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
@ -133,18 +134,13 @@ public class TestEventFlow {
ApplicationAttemptId.newInstance(applicationId, 0); ApplicationAttemptId.newInstance(applicationId, 0);
ContainerId cID = ContainerId.newInstance(applicationAttemptId, 0); ContainerId cID = ContainerId.newInstance(applicationAttemptId, 0);
Resource r = BuilderUtils.newResource(1024, 1);
String user = "testing"; String user = "testing";
String host = "127.0.0.1";
int port = 1234;
Token containerToken =
BuilderUtils.newContainerToken(cID, host, port, user, r,
System.currentTimeMillis() + 10000L, 123, "password".getBytes(),
SIMULATED_RM_IDENTIFIER);
StartContainerRequest request = StartContainerRequest request =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
request.setContainerLaunchContext(launchContext); request.setContainerLaunchContext(launchContext);
request.setContainerToken(containerToken); request.setContainerToken(TestContainerManager.createContainerToken(cID,
SIMULATED_RM_IDENTIFIER, context.getNodeId(), user,
context.getContainerTokenSecretManager()));
containerManager.startContainer(request); containerManager.startContainer(request);
BaseContainerManagerTest.waitForContainerState(containerManager, cID, BaseContainerManagerTest.waitForContainerState(containerManager, cID,

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.yarn.server.nodemanager; package org.apache.hadoop.yarn.server.nodemanager;
import static org.mockito.Matchers.argThat; import static org.mockito.Matchers.argThat;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isNull; import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.spy; import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times; import static org.mockito.Mockito.times;
@ -50,17 +49,16 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState; import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.ContainerState;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.junit.After; import org.junit.After;
@ -131,24 +129,23 @@ public class TestNodeManagerReboot {
containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setLocalResources(localResources);
List<String> commands = new ArrayList<String>(); List<String> commands = new ArrayList<String>();
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource resource = Records.newRecord(Resource.class);
resource.setMemory(1024);
NodeId nodeId = BuilderUtils.newNodeId("127.0.0.1", 12345);
Token containerToken =
BuilderUtils.newContainerToken(cId, nodeId.getHost(), nodeId.getPort(),
user, resource, System.currentTimeMillis() + 10000L, 123,
"password".getBytes(), 0);
final StartContainerRequest startRequest = final StartContainerRequest startRequest =
Records.newRecord(StartContainerRequest.class); Records.newRecord(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(containerToken); NodeId nodeId = nm.getNMContext().getNodeId();
startRequest.setContainerToken(TestContainerManager.createContainerToken(
cId, 0, nodeId, destinationFile, nm.getNMContext()
.getContainerTokenSecretManager()));
final UserGroupInformation currentUser = UserGroupInformation final UserGroupInformation currentUser = UserGroupInformation
.createRemoteUser(cId.toString()); .createRemoteUser(cId.getApplicationAttemptId().toString());
NMTokenIdentifier nmIdentifier =
new NMTokenIdentifier(cId.getApplicationAttemptId(), nodeId, user, 123);
currentUser.addTokenIdentifier(nmIdentifier);
currentUser.doAs(new PrivilegedExceptionAction<Void>() { currentUser.doAs(new PrivilegedExceptionAction<Void>() {
@Override @Override
public Void run() throws YarnException, IOException { public Void run() throws YarnException, IOException {
containerManager.startContainer(startRequest); nm.getContainerManager().startContainer(startRequest);
return null; return null;
} }
}); });
@ -208,8 +205,6 @@ public class TestNodeManagerReboot {
ContainerLocalizer.FILECACHE) == 0 && numOfLocalDirs(nmLocalDir ContainerLocalizer.FILECACHE) == 0 && numOfLocalDirs(nmLocalDir
.getAbsolutePath(), ResourceLocalizationService.NM_PRIVATE_DIR) .getAbsolutePath(), ResourceLocalizationService.NM_PRIVATE_DIR)
== 0); == 0);
verify(delService, times(1)).delete(eq(user),
argThat(new PathInclude(user)));
verify(delService, times(1)).delete( verify(delService, times(1)).delete(
(String) isNull(), (String) isNull(),
argThat(new PathInclude(ResourceLocalizationService.NM_PRIVATE_DIR argThat(new PathInclude(ResourceLocalizationService.NM_PRIVATE_DIR

View File

@ -184,6 +184,7 @@ public class TestNodeManagerResync {
} catch (InterruptedException e) { } catch (InterruptedException e) {
} catch (BrokenBarrierException e) { } catch (BrokenBarrierException e) {
} catch (AssertionError ae) { } catch (AssertionError ae) {
ae.printStackTrace();
assertionFailedInThread.set(true); assertionFailedInThread.set(true);
} }
} }
@ -228,6 +229,7 @@ public class TestNodeManagerResync {
.setStopThreadFlag(false); .setStopThreadFlag(false);
super.setBlockNewContainerRequests(blockNewContainerRequests); super.setBlockNewContainerRequests(blockNewContainerRequests);
} catch (InterruptedException e) { } catch (InterruptedException e) {
e.printStackTrace();
} }
} }
} }
@ -258,6 +260,7 @@ public class TestNodeManagerResync {
} catch (InterruptedException e) { } catch (InterruptedException e) {
} catch (BrokenBarrierException e) { } catch (BrokenBarrierException e) {
} catch (AssertionError ae) { } catch (AssertionError ae) {
ae.printStackTrace();
assertionFailedInThread.set(true); assertionFailedInThread.set(true);
} }
} }
@ -296,6 +299,7 @@ public class TestNodeManagerResync {
Assert.assertEquals(NMNotYetReadyException.class.getName(), e Assert.assertEquals(NMNotYetReadyException.class.getName(), e
.getClass().getName()); .getClass().getName());
} catch (IOException e) { } catch (IOException e) {
e.printStackTrace();
assertionFailedInThread.set(true); assertionFailedInThread.set(true);
} }
} }

View File

@ -52,16 +52,17 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.Dispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.After; import org.junit.After;
@ -161,7 +162,7 @@ public class TestNodeManagerShutdown {
ContainerLaunchContext containerLaunchContext = ContainerLaunchContext containerLaunchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class); recordFactory.newRecordInstance(ContainerLaunchContext.class);
NodeId nodeId = BuilderUtils.newNodeId("localhost", 1234); NodeId nodeId = BuilderUtils.newNodeId("localhost", 12345);
URL localResourceUri = URL localResourceUri =
ConverterUtils.getYarnUrlFromPath(localFS ConverterUtils.getYarnUrlFromPath(localFS
@ -180,17 +181,22 @@ public class TestNodeManagerShutdown {
containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setLocalResources(localResources);
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile)); List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource resource = BuilderUtils.newResource(1024, 1);
Token containerToken =
BuilderUtils.newContainerToken(cId, nodeId.getHost(), nodeId.getPort(),
user, resource, System.currentTimeMillis() + 10000L, 123,
"password".getBytes(), 0);
StartContainerRequest startRequest = StartContainerRequest startRequest =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(containerToken); startRequest
.setContainerToken(TestContainerManager.createContainerToken(cId, 0,
nodeId, user, nm.getNMContext().getContainerTokenSecretManager()));
final InetSocketAddress containerManagerBindAddress =
NetUtils.createSocketAddrForHost("127.0.0.1", 12345);
UserGroupInformation currentUser = UserGroupInformation UserGroupInformation currentUser = UserGroupInformation
.createRemoteUser(cId.toString()); .createRemoteUser(cId.toString());
org.apache.hadoop.security.token.Token<NMTokenIdentifier> nmToken =
ConverterUtils.convertFromYarn(
nm.getNMContext().getNMTokenSecretManager()
.createNMToken(cId.getApplicationAttemptId(), nodeId, user),
containerManagerBindAddress);
currentUser.addToken(nmToken);
ContainerManagementProtocol containerManager = ContainerManagementProtocol containerManager =
currentUser.doAs(new PrivilegedAction<ContainerManagementProtocol>() { currentUser.doAs(new PrivilegedAction<ContainerManagementProtocol>() {

View File

@ -201,7 +201,7 @@ public class TestNodeStatusUpdater {
String user = "testUser"; String user = "testUser";
ContainerTokenIdentifier containerToken = ContainerTokenIdentifier containerToken =
BuilderUtils.newContainerTokenIdentifier(BuilderUtils BuilderUtils.newContainerTokenIdentifier(BuilderUtils
.newContainerToken(firstContainerID, "127.0.0.1", 1234, user, .newContainerToken(firstContainerID, "localhost", 1234, user,
resource, currentTime + 10000, 123, "password".getBytes(), resource, currentTime + 10000, 123, "password".getBytes(),
currentTime)); currentTime));
Container container = Container container =
@ -232,7 +232,7 @@ public class TestNodeStatusUpdater {
Resource resource = BuilderUtils.newResource(3, 1); Resource resource = BuilderUtils.newResource(3, 1);
ContainerTokenIdentifier containerToken = ContainerTokenIdentifier containerToken =
BuilderUtils.newContainerTokenIdentifier(BuilderUtils BuilderUtils.newContainerTokenIdentifier(BuilderUtils
.newContainerToken(secondContainerID, "127.0.0.1", 1234, user, .newContainerToken(secondContainerID, "localhost", 1234, user,
resource, currentTime + 10000, 123, resource, currentTime + 10000, 123,
"password".getBytes(), currentTime)); "password".getBytes(), currentTime));
Container container = Container container =
@ -1168,8 +1168,8 @@ public class TestNodeStatusUpdater {
private YarnConfiguration createNMConfig() { private YarnConfiguration createNMConfig() {
YarnConfiguration conf = new YarnConfiguration(); YarnConfiguration conf = new YarnConfiguration();
conf.setInt(YarnConfiguration.NM_PMEM_MB, 5*1024); // 5GB conf.setInt(YarnConfiguration.NM_PMEM_MB, 5*1024); // 5GB
conf.set(YarnConfiguration.NM_ADDRESS, "127.0.0.1:12345"); conf.set(YarnConfiguration.NM_ADDRESS, "localhost:12345");
conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "127.0.0.1:12346"); conf.set(YarnConfiguration.NM_LOCALIZER_ADDRESS, "localhost:12346");
conf.set(YarnConfiguration.NM_LOG_DIRS, new Path(basedir, "logs").toUri() conf.set(YarnConfiguration.NM_LOG_DIRS, new Path(basedir, "logs").toUri()
.getPath()); .getPath());
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, new Path(basedir, conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, new Path(basedir,

View File

@ -30,21 +30,20 @@ import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.service.Service.STATE; import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.ContainerState; import org.apache.hadoop.yarn.api.records.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.ResourceTracker; import org.apache.hadoop.yarn.server.api.ResourceTracker;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.Context;
@ -58,6 +57,7 @@ import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdater;
import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl; import org.apache.hadoop.yarn.server.nodemanager.NodeStatusUpdaterImpl;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Application;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
@ -150,7 +150,7 @@ public abstract class BaseContainerManagerTest {
LOG.info("Created localDir in " + localDir.getAbsolutePath()); LOG.info("Created localDir in " + localDir.getAbsolutePath());
LOG.info("Created tmpDir in " + tmpDir.getAbsolutePath()); LOG.info("Created tmpDir in " + tmpDir.getAbsolutePath());
String bindAddress = "0.0.0.0:5555"; String bindAddress = "127.0.0.1:12345";
conf.set(YarnConfiguration.NM_ADDRESS, bindAddress); conf.set(YarnConfiguration.NM_ADDRESS, bindAddress);
conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath()); conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath()); conf.set(YarnConfiguration.NM_LOG_DIRS, localLogDir.getAbsolutePath());
@ -173,6 +173,7 @@ public abstract class BaseContainerManagerTest {
protected ContainerManagerImpl protected ContainerManagerImpl
createContainerManager(DeletionService delSrvc) { createContainerManager(DeletionService delSrvc) {
return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater, return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler) { metrics, new ApplicationACLsManager(conf), dirsHandler) {
@Override @Override
@ -182,11 +183,24 @@ public abstract class BaseContainerManagerTest {
} }
@Override @Override
protected void authorizeRequest(String containerIDStr, protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
ContainerLaunchContext launchContext, UserGroupInformation remoteUgi, Container container, boolean stopRequest) throws YarnException {
ContainerTokenIdentifier tokenId) throws YarnException { // do nothing
// do nothing }
}
@Override
protected void authorizeStartRequest(
NMTokenIdentifier nmTokenIdentifier,
ContainerTokenIdentifier containerTokenIdentifier,
UserGroupInformation ugi) throws YarnException {
// do nothing
}
@Override
protected void updateNMTokenIdentifier(
NMTokenIdentifier nmTokenIdentifier) throws InvalidToken {
// Do nothing
}
}; };
} }
@ -242,7 +256,7 @@ public abstract class BaseContainerManagerTest {
throws InterruptedException { throws InterruptedException {
// Wait for app-finish // Wait for app-finish
Application app = Application app =
containerManager.context.getApplications().get(appID); containerManager.getContext().getApplications().get(appID);
int timeout = 0; int timeout = 0;
while (!(app.getApplicationState().equals(finalState)) while (!(app.getApplicationState().equals(finalState))
&& timeout++ < 15) { && timeout++ < 15) {

View File

@ -34,6 +34,7 @@ import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.fs.FileContext; import org.apache.hadoop.fs.FileContext;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell;
import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
@ -47,10 +48,13 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.ResourceManagerConstants; import org.apache.hadoop.yarn.server.api.ResourceManagerConstants;
import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent; import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
@ -59,8 +63,11 @@ import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService; import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ResourceLocalizationService;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.server.utils.BuilderUtils;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.Before;
import org.junit.Test; import org.junit.Test;
public class TestContainerManager extends BaseContainerManagerTest { public class TestContainerManager extends BaseContainerManagerTest {
@ -72,6 +79,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
static { static {
LOG = LogFactory.getLog(TestContainerManager.class); LOG = LogFactory.getLog(TestContainerManager.class);
} }
@Override
@Before
public void setup() throws IOException {
super.setup();
}
private ContainerId createContainerId() { private ContainerId createContainerId() {
ApplicationId appId = ApplicationId.newInstance(0, 0); ApplicationId appId = ApplicationId.newInstance(0, 0);
@ -81,6 +94,32 @@ public class TestContainerManager extends BaseContainerManagerTest {
return containerId; return containerId;
} }
@Override
protected ContainerManagerImpl
createContainerManager(DeletionService delSrvc) {
return new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler) {
@Override
public void
setBlockNewContainerRequests(boolean blockNewContainerRequests) {
// do nothing
}
@Override
protected UserGroupInformation getRemoteUgi() throws YarnException {
ApplicationId appId = ApplicationId.newInstance(0, 0);
ApplicationAttemptId appAttemptId =
ApplicationAttemptId.newInstance(appId, 1);
UserGroupInformation ugi =
UserGroupInformation.createRemoteUser(appAttemptId.toString());
ugi.addTokenIdentifier(new NMTokenIdentifier(appAttemptId, context
.getNodeId(), user, context.getNMTokenSecretManager().getCurrentKey()
.getKeyId()));
return ugi;
}
};
}
@Test @Test
public void testContainerManagerInitialization() throws IOException { public void testContainerManagerInitialization() throws IOException {
@ -101,8 +140,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
} }
@Test @Test
public void testContainerSetup() throws IOException, InterruptedException, public void testContainerSetup() throws Exception {
YarnException {
containerManager.start(); containerManager.start();
@ -134,16 +172,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
new HashMap<String, LocalResource>(); new HashMap<String, LocalResource>();
localResources.put(destinationFile, rsrc_alpha); localResources.put(destinationFile, rsrc_alpha);
containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setLocalResources(localResources);
Resource r = BuilderUtils.newResource(512, 1);
int port = 12345;
Token containerToken =
BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
port, user, r, System.currentTimeMillis() + 10000L, 123,
"password".getBytes(), super.DUMMY_RM_IDENTIFIER);
StartContainerRequest startRequest = StartContainerRequest startRequest =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(containerToken); startRequest.setContainerToken(createContainerToken(cId,
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
context.getContainerTokenSecretManager()));
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
@ -227,16 +261,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setLocalResources(localResources);
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile)); List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource r = BuilderUtils.newResource(100, 1);
int port = 12345;
Token containerToken =
BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
port, user, r, System.currentTimeMillis() + 10000L, 123,
"password".getBytes(), super.DUMMY_RM_IDENTIFIER);
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class); StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(containerToken); startRequest.setContainerToken(createContainerToken(cId,
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
context.getContainerTokenSecretManager()));
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
int timeoutSecs = 0; int timeoutSecs = 0;
@ -335,15 +365,12 @@ public class TestContainerManager extends BaseContainerManagerTest {
containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setLocalResources(localResources);
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile)); List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource r = BuilderUtils.newResource(100, 1);
int port = 12345;
Token containerToken =
BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
port, user, r, System.currentTimeMillis() + 10000L, 123,
"password".getBytes(), super.DUMMY_RM_IDENTIFIER);
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class); StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(containerToken); startRequest.setContainerToken(createContainerToken(cId,
DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
context.getContainerTokenSecretManager()));
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
BaseContainerManagerTest.waitForContainerState(containerManager, cId, BaseContainerManagerTest.waitForContainerState(containerManager, cId,
@ -423,16 +450,10 @@ public class TestContainerManager extends BaseContainerManagerTest {
new HashMap<String, LocalResource>(); new HashMap<String, LocalResource>();
localResources.put(destinationFile, rsrc_alpha); localResources.put(destinationFile, rsrc_alpha);
containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setLocalResources(localResources);
Resource r = BuilderUtils.newResource(100, 1);
int port = 12345;
Token containerToken =
BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
port, user, r, System.currentTimeMillis() + 10000L, 123,
"password".getBytes(), super.DUMMY_RM_IDENTIFIER);
StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class); StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
request.setContainerLaunchContext(containerLaunchContext); request.setContainerLaunchContext(containerLaunchContext);
request.setContainerToken(containerToken); request.setContainerToken(createContainerToken(cId, DUMMY_RM_IDENTIFIER,
context.getNodeId(), user, context.getContainerTokenSecretManager()));
containerManager.startContainer(request); containerManager.startContainer(request);
BaseContainerManagerTest.waitForContainerState(containerManager, cId, BaseContainerManagerTest.waitForContainerState(containerManager, cId,
@ -503,24 +524,19 @@ public class TestContainerManager extends BaseContainerManagerTest {
ContainerLaunchContext containerLaunchContext = ContainerLaunchContext containerLaunchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class); recordFactory.newRecordInstance(ContainerLaunchContext.class);
String host = "127.0.0.1";
int port = 1234;
ContainerId cId1 = createContainerId(); ContainerId cId1 = createContainerId();
ContainerId cId2 = createContainerId(); ContainerId cId2 = createContainerId();
containerLaunchContext containerLaunchContext
.setLocalResources(new HashMap<String, LocalResource>()); .setLocalResources(new HashMap<String, LocalResource>());
Resource mockResource = BuilderUtils.newResource(1024, 1);
// Construct the Container with Invalid RMIdentifier // Construct the Container with Invalid RMIdentifier
StartContainerRequest startRequest1 = StartContainerRequest startRequest1 =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest1.setContainerLaunchContext(containerLaunchContext); startRequest1.setContainerLaunchContext(containerLaunchContext);
Token containerToken1 = startRequest1.setContainerToken(createContainerToken(cId1,
BuilderUtils.newContainerToken(cId1, host, port, user, mockResource, ResourceManagerConstants.RM_INVALID_IDENTIFIER, context.getNodeId(),
System.currentTimeMillis() + 10000, 123, "password".getBytes(), user, context.getContainerTokenSecretManager()));
(long) ResourceManagerConstants.RM_INVALID_IDENTIFIER);
startRequest1.setContainerToken(containerToken1);
boolean catchException = false; boolean catchException = false;
try { try {
containerManager.startContainer(startRequest1); containerManager.startContainer(startRequest1);
@ -528,8 +544,8 @@ public class TestContainerManager extends BaseContainerManagerTest {
catchException = true; catchException = true;
Assert.assertTrue(e.getMessage().contains( Assert.assertTrue(e.getMessage().contains(
"Container " + cId1 + " rejected as it is allocated by a previous RM")); "Container " + cId1 + " rejected as it is allocated by a previous RM"));
Assert.assertEquals(InvalidContainerException.class.getName(), e Assert.assertTrue(e.getClass().getName()
.getClass().getName()); .equalsIgnoreCase(InvalidContainerException.class.getName()));
} }
// Verify that startContainer fail because of invalid container request // Verify that startContainer fail because of invalid container request
@ -539,11 +555,9 @@ public class TestContainerManager extends BaseContainerManagerTest {
StartContainerRequest startRequest2 = StartContainerRequest startRequest2 =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest2.setContainerLaunchContext(containerLaunchContext); startRequest2.setContainerLaunchContext(containerLaunchContext);
Token containerToken2 = startRequest2.setContainerToken(createContainerToken(cId2,
BuilderUtils.newContainerToken(cId1, host, port, user, mockResource, DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
System.currentTimeMillis() + 10000, 123, "password".getBytes(), context.getContainerTokenSecretManager()));
super.DUMMY_RM_IDENTIFIER);
startRequest2.setContainerToken(containerToken2);
boolean noException = true; boolean noException = true;
try { try {
containerManager.startContainer(startRequest2); containerManager.startContainer(startRequest2);
@ -553,4 +567,20 @@ public class TestContainerManager extends BaseContainerManagerTest {
// Verify that startContainer get no YarnException // Verify that startContainer get no YarnException
Assert.assertTrue(noException); Assert.assertTrue(noException);
} }
public static Token createContainerToken(ContainerId cId, long rmIdentifier,
NodeId nodeId, String user,
NMContainerTokenSecretManager containerTokenSecretManager)
throws IOException {
Resource r = BuilderUtils.newResource(1024, 1);
ContainerTokenIdentifier containerTokenIdentifier =
new ContainerTokenIdentifier(cId, nodeId.toString(), user, r,
System.currentTimeMillis() + 100000L, 123, rmIdentifier);
Token containerToken =
BuilderUtils
.newContainerToken(nodeId, containerTokenSecretManager
.retrievePassword(containerTokenIdentifier),
containerTokenIdentifier);
return containerToken;
}
} }

View File

@ -37,6 +37,7 @@ import junit.framework.Assert;
import org.apache.hadoop.fs.FileUtil; import org.apache.hadoop.fs.FileUtil;
import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.security.token.SecretManager.InvalidToken;
import org.apache.hadoop.util.Shell; import org.apache.hadoop.util.Shell;
import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.api.ApplicationConstants.Environment; import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
@ -56,6 +57,7 @@ import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
@ -229,14 +231,9 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
// set up the rest of the container // set up the rest of the container
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile)); List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource r = BuilderUtils.newResource(1024, 1);
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class); StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
Token containerToken = startRequest.setContainerToken(createContainerToken(cId));
BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(),
port, user, r, System.currentTimeMillis() + 10000L, 1234,
"password".getBytes(), super.DUMMY_RM_IDENTIFIER);
startRequest.setContainerToken(containerToken);
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
int timeoutSecs = 0; int timeoutSecs = 0;
@ -378,12 +375,9 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
// set up the rest of the container // set up the rest of the container
List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile)); List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource r = BuilderUtils.newResource(1024, 1); Token containerToken = createContainerToken(cId);
Token containerToken = StartContainerRequest startRequest =
BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(), recordFactory.newRecordInstance(StartContainerRequest.class);
port, user, r, System.currentTimeMillis() + 10000L, 123,
"password".getBytes(), super.DUMMY_RM_IDENTIFIER);
StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(containerToken); startRequest.setContainerToken(containerToken);
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
@ -441,4 +435,17 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
} }
} }
protected Token createContainerToken(ContainerId cId) throws InvalidToken {
Resource r = BuilderUtils.newResource(1024, 1);
ContainerTokenIdentifier containerTokenIdentifier =
new ContainerTokenIdentifier(cId, context.getNodeId().toString(), user,
r, System.currentTimeMillis() + 10000L, 123, DUMMY_RM_IDENTIFIER);
Token containerToken =
BuilderUtils.newContainerToken(
context.getNodeId(),
context.getContainerTokenSecretManager().retrievePassword(
containerTokenIdentifier), containerTokenIdentifier);
return containerToken;
}
} }

View File

@ -23,9 +23,9 @@ import static junit.framework.Assert.assertTrue;
import static org.mockito.Matchers.any; import static org.mockito.Matchers.any;
import static org.mockito.Matchers.anyMap; import static org.mockito.Matchers.anyMap;
import static org.mockito.Matchers.eq; import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isA;
import static org.mockito.Mockito.atLeast; import static org.mockito.Mockito.atLeast;
import static org.mockito.Mockito.doThrow; import static org.mockito.Mockito.doThrow;
import static org.mockito.Mockito.isA;
import static org.mockito.Mockito.mock; import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.never; import static org.mockito.Mockito.never;
import static org.mockito.Mockito.reset; import static org.mockito.Mockito.reset;
@ -73,8 +73,6 @@ import org.apache.hadoop.yarn.api.records.LocalResource;
import org.apache.hadoop.yarn.api.records.LocalResourceType; import org.apache.hadoop.yarn.api.records.LocalResourceType;
import org.apache.hadoop.yarn.api.records.LocalResourceVisibility; import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.URL; import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher;
@ -94,6 +92,7 @@ import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService; import org.apache.hadoop.yarn.server.nodemanager.LocalDirsHandlerService;
import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext; import org.apache.hadoop.yarn.server.nodemanager.NodeManager.NMContext;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest; import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.TestContainerManager;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerAppFinishedEvent;
@ -810,15 +809,12 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
commands.add("/bin/bash"); commands.add("/bin/bash");
commands.add(scriptFile.getAbsolutePath()); commands.add(scriptFile.getAbsolutePath());
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource r = BuilderUtils.newResource(100 * 1024 * 1024, 1);
Token containerToken =
BuilderUtils.newContainerToken(cId, "127.0.0.1", 1234, user, r,
System.currentTimeMillis() + 10000L, 123, "password".getBytes(),
super.DUMMY_RM_IDENTIFIER);
StartContainerRequest startRequest = StartContainerRequest startRequest =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainerToken(containerToken); startRequest.setContainerToken(TestContainerManager.createContainerToken(
cId, DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
context.getContainerTokenSecretManager()));
this.containerManager.startContainer(startRequest); this.containerManager.startContainer(startRequest);
BaseContainerManagerTest.waitForContainerState(this.containerManager, BaseContainerManagerTest.waitForContainerState(this.containerManager,

View File

@ -57,6 +57,7 @@ import org.apache.hadoop.yarn.api.records.URL;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.event.AsyncDispatcher; import org.apache.hadoop.yarn.event.AsyncDispatcher;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal; import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
@ -229,13 +230,16 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
StartContainerRequest startRequest = StartContainerRequest startRequest =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
ContainerTokenIdentifier containerIdentifier =
new ContainerTokenIdentifier(cId, context.getNodeId().toString(), user,
r, System.currentTimeMillis() + 120000, 123, DUMMY_RM_IDENTIFIER);
Token containerToken = Token containerToken =
BuilderUtils.newContainerToken(cId, context.getNodeId().getHost(), BuilderUtils.newContainerToken(context.getNodeId(),
port, user, r, System.currentTimeMillis() + 10000L, 123, containerManager.getContext().getContainerTokenSecretManager()
"password".getBytes(), super.DUMMY_RM_IDENTIFIER); .createPassword(containerIdentifier), containerIdentifier);
startRequest.setContainerToken(containerToken); startRequest.setContainerToken(containerToken);
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
int timeoutSecs = 0; int timeoutSecs = 0;
while (!processStartFile.exists() && timeoutSecs++ < 20) { while (!processStartFile.exists() && timeoutSecs++ < 20) {
Thread.sleep(1000); Thread.sleep(1000);

View File

@ -0,0 +1,25 @@
<?xml version="1.0" encoding="UTF-8"?>
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
<!--
Licensed 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. See accompanying LICENSE file.
-->
<!-- Put site-specific property overrides in this file. -->
<configuration>
<property>
<name>hadoop.security.token.service.use_ip</name>
<value>false</value>
</property>
</configuration>

View File

@ -380,7 +380,7 @@ public class ApplicationMasterService extends AbstractService implements
// Adding NMTokens for allocated containers. // Adding NMTokens for allocated containers.
if (!allocation.getContainers().isEmpty()) { if (!allocation.getContainers().isEmpty()) {
allocateResponse.setNMTokens(rmContext.getNMTokenSecretManager() allocateResponse.setNMTokens(rmContext.getNMTokenSecretManager()
.getNMTokens(app.getUser(), appAttemptId, .createAndGetNMTokens(app.getUser(), appAttemptId,
allocation.getContainers())); allocation.getContainers()));
} }
return allocateResponse; return allocateResponse;

View File

@ -131,21 +131,30 @@ public class AMLauncher implements Runnable {
final YarnRPC rpc = YarnRPC.create(conf); // TODO: Don't create again and again. final YarnRPC rpc = YarnRPC.create(conf); // TODO: Don't create again and again.
UserGroupInformation currentUser = UserGroupInformation UserGroupInformation currentUser =
.createRemoteUser(containerId.toString()); UserGroupInformation.createRemoteUser(containerId
if (UserGroupInformation.isSecurityEnabled()) { .getApplicationAttemptId().toString());
Token<ContainerTokenIdentifier> token =
ConverterUtils.convertFromYarn(masterContainer String user =
.getContainerToken(), containerManagerBindAddress); rmContext.getRMApps()
currentUser.addToken(token); .get(containerId.getApplicationAttemptId().getApplicationId())
} .getUser();
return currentUser.doAs(new PrivilegedAction<ContainerManagementProtocol>() { org.apache.hadoop.yarn.api.records.Token token =
@Override rmContext.getNMTokenSecretManager().createNMToken(
public ContainerManagementProtocol run() { containerId.getApplicationAttemptId(), node, user);
return (ContainerManagementProtocol) rpc.getProxy(ContainerManagementProtocol.class, currentUser.addToken(ConverterUtils.convertFromYarn(token,
containerManagerBindAddress, conf); containerManagerBindAddress));
}
}); return currentUser
.doAs(new PrivilegedAction<ContainerManagementProtocol>() {
@Override
public ContainerManagementProtocol run() {
return (ContainerManagementProtocol) rpc.getProxy(
ContainerManagementProtocol.class,
containerManagerBindAddress, conf);
}
});
} }
private ContainerLaunchContext createAMContainerLaunchContext( private ContainerLaunchContext createAMContainerLaunchContext(
@ -234,7 +243,13 @@ public class AMLauncher implements Runnable {
} catch(IOException ie) { } catch(IOException ie) {
LOG.info("Error cleaning master ", ie); LOG.info("Error cleaning master ", ie);
} catch (YarnException e) { } catch (YarnException e) {
LOG.info("Error cleaning master ", e); StringBuilder sb = new StringBuilder("Container ");
sb.append(masterContainer.getId().toString());
sb.append(" is not handled by this NodeManager");
if (!e.getMessage().contains(sb.toString())) {
// Ignoring if container is already killed by Node Manager.
LOG.info("Error cleaning master ", e);
}
} }
break; break;
default: default:

View File

@ -18,7 +18,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.security; package org.apache.hadoop.yarn.server.resourcemanager.security;
import java.net.InetSocketAddress;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashSet; import java.util.HashSet;
import java.util.Iterator; import java.util.Iterator;
@ -31,16 +30,12 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.classification.InterfaceAudience.Private; import org.apache.hadoop.classification.InterfaceAudience.Private;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.records.Container; import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.NMToken; import org.apache.hadoop.yarn.api.records.NMToken;
import org.apache.hadoop.yarn.api.records.NodeId; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Token; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.impl.pb.NMTokenPBImpl;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.NMTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager; import org.apache.hadoop.yarn.server.security.BaseNMTokenSecretManager;
import org.apache.hadoop.yarn.server.security.MasterKeyData; import org.apache.hadoop.yarn.server.security.MasterKeyData;
@ -183,7 +178,7 @@ public class NMTokenSecretManagerInRM extends BaseNMTokenSecretManager {
} }
} }
public List<NMToken> getNMTokens(String applicationSubmitter, public List<NMToken> createAndGetNMTokens(String applicationSubmitter,
ApplicationAttemptId appAttemptId, List<Container> containers) { ApplicationAttemptId appAttemptId, List<Container> containers) {
try { try {
this.readLock.lock(); this.readLock.lock();
@ -193,12 +188,14 @@ public class NMTokenSecretManagerInRM extends BaseNMTokenSecretManager {
for (Container container : containers) { for (Container container : containers) {
if (!nodeSet.contains(container.getNodeId())) { if (!nodeSet.contains(container.getNodeId())) {
LOG.debug("Sending NMToken for nodeId : " LOG.debug("Sending NMToken for nodeId : "
+ container.getNodeId().toString()); + container.getNodeId().toString()
+ " for application attempt : " + appAttemptId.toString());
Token token = createNMToken(appAttemptId, container.getNodeId(), Token token = createNMToken(appAttemptId, container.getNodeId(),
applicationSubmitter); applicationSubmitter);
NMToken nmToken = NMToken nmToken =
NMToken.newInstance(container.getNodeId(), token); NMToken.newInstance(container.getNodeId(), token);
nmTokens.add(nmToken); nmTokens.add(nmToken);
// This will update the nmToken set.
nodeSet.add(container.getNodeId()); nodeSet.add(container.getNodeId());
} }
} }
@ -273,38 +270,4 @@ public class NMTokenSecretManagerInRM extends BaseNMTokenSecretManager {
this.writeLock.unlock(); this.writeLock.unlock();
} }
} }
public static Token newNMToken(byte[] password,
NMTokenIdentifier identifier) {
NodeId nodeId = identifier.getNodeId();
// RPC layer client expects ip:port as service for tokens
InetSocketAddress addr =
NetUtils.createSocketAddrForHost(nodeId.getHost(), nodeId.getPort());
Token nmToken =
Token.newInstance(identifier.getBytes(),
NMTokenIdentifier.KIND.toString(), password, SecurityUtil
.buildTokenService(addr).toString());
return nmToken;
}
/**
* Helper function for creating NMTokens.
*/
public Token createNMToken(ApplicationAttemptId applicationAttemptId,
NodeId nodeId, String applicationSubmitter) {
byte[] password;
NMTokenIdentifier identifier;
this.readLock.lock();
try {
identifier =
new NMTokenIdentifier(applicationAttemptId, nodeId,
applicationSubmitter, this.currentMasterKey.getMasterKey()
.getKeyId());
password = this.createPassword(identifier);
} finally {
this.readLock.unlock();
}
return newNMToken(password, identifier);
}
} }

View File

@ -59,9 +59,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEvent;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeEventType;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl; import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNodeImpl;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMDelegationTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.apache.log4j.Level; import org.apache.log4j.Level;
import org.apache.log4j.LogManager; import org.apache.log4j.LogManager;
@ -310,6 +308,7 @@ public class MockRM extends ResourceManager {
Configuration conf = new Configuration(); Configuration conf = new Configuration();
containerTokenSecretManager.rollMasterKey(); containerTokenSecretManager.rollMasterKey();
nmTokenSecretManager.rollMasterKey();
return new ResourceTrackerService(getRMContext(), nodesListManager, return new ResourceTrackerService(getRMContext(), nodesListManager,
this.nmLivelinessMonitor, containerTokenSecretManager, this.nmLivelinessMonitor, containerTokenSecretManager,
nmTokenSecretManager) { nmTokenSecretManager) {

View File

@ -22,14 +22,7 @@ import static org.junit.Assert.fail;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.security.PrivilegedAction; import java.security.PrivilegedAction;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import junit.framework.Assert; import junit.framework.Assert;
@ -37,45 +30,30 @@ import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory; import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.SecurityUtil;
import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.util.Shell;
import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
import org.apache.hadoop.yarn.api.ContainerManagementProtocol; import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest; import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
import org.apache.hadoop.yarn.api.protocolrecords.KillApplicationRequest;
import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest; import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationId; import org.apache.hadoop.yarn.api.records.ApplicationId;
import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
import org.apache.hadoop.yarn.api.records.Container;
import org.apache.hadoop.yarn.api.records.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.api.records.LocalResource; import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.Token;
import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
import org.apache.hadoop.yarn.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.exceptions.YarnException; import org.apache.hadoop.yarn.exceptions.YarnException;
import org.apache.hadoop.yarn.factories.RecordFactory; import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider; import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.ipc.YarnRPC; import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.security.AMRMTokenIdentifier; import org.apache.hadoop.yarn.server.nodemanager.NodeManager;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier; import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMTokenSecretManagerInNM;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp; import org.apache.hadoop.yarn.server.resourcemanager.security.NMTokenSecretManagerInRM;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptState;
import org.apache.hadoop.yarn.server.resourcemanager.security.AMRMTokenSecretManager;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.utils.BuilderUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.junit.Test; import org.junit.Test;
@ -111,15 +89,15 @@ public class TestContainerManagerSecurity {
yarnCluster.init(conf); yarnCluster.init(conf);
yarnCluster.start(); yarnCluster.start();
// Testing for authenticated user // TestNMTokens.
testAuthenticatedUser(); testNMTokens(conf);
// Testing for malicious user // Testing for container token tampering
testMaliceUser(); testContainerToken(conf);
// Testing for usage of expired tokens
testExpiredTokens();
} catch (Exception e) {
e.printStackTrace();
throw e;
} finally { } finally {
if (yarnCluster != null) { if (yarnCluster != null) {
yarnCluster.stop(); yarnCluster.stop();
@ -128,57 +106,264 @@ public class TestContainerManagerSecurity {
} }
} }
private void testAuthenticatedUser() throws IOException, private void testNMTokens(Configuration conf) throws Exception {
InterruptedException, YarnException { NMTokenSecretManagerInRM nmTokenSecretManagerRM =
yarnCluster.getResourceManager().getRMContext()
.getNMTokenSecretManager();
NMTokenSecretManagerInNM nmTokenSecretManagerNM =
yarnCluster.getNodeManager(0).getNMContext().getNMTokenSecretManager();
RMContainerTokenSecretManager containerTokenSecretManager =
yarnCluster.getResourceManager().getRMContainerTokenSecretManager();
NodeManager nm = yarnCluster.getNodeManager(0);
waitForNMToReceiveNMTokenKey(nmTokenSecretManagerNM, nm);
// Both id should be equal.
Assert.assertEquals(nmTokenSecretManagerNM.getCurrentKey().getKeyId(),
nmTokenSecretManagerRM.getCurrentKey().getKeyId());
/*
* Below cases should be tested.
* 1) If Invalid NMToken is used then it should be rejected.
* 2) If valid NMToken but belonging to another Node is used then that
* too should be rejected.
* 3) NMToken for say appAttempt-1 is used for starting/stopping/retrieving
* status for container with containerId for say appAttempt-2 should
* be rejected.
* 4) After start container call is successful nmtoken should have been
* saved in NMTokenSecretManagerInNM.
* 5) If start container call was successful (no matter if container is
* still running or not), appAttempt->NMToken should be present in
* NMTokenSecretManagerInNM's cache. Any future getContainerStatus call
* for containerId belonging to that application attempt using
* applicationAttempt's older nmToken should not get any invalid
* nmToken error. (This can be best tested if we roll over NMToken
* master key twice).
*/
YarnRPC rpc = YarnRPC.create(conf);
String user = "test";
Resource r = Resource.newInstance(1024, 1);
LOG.info("Running test for authenticated user"); ApplicationId appId = ApplicationId.newInstance(1, 1);
ApplicationAttemptId validAppAttemptId =
ApplicationAttemptId.newInstance(appId, 1);
ApplicationAttemptId invalidAppAttemptId =
ApplicationAttemptId.newInstance(appId, 2);
ContainerId validContainerId =
ContainerId.newInstance(validAppAttemptId, 0);
NodeId validNode = yarnCluster.getNodeManager(0).getNMContext().getNodeId();
NodeId invalidNode = NodeId.newInstance("InvalidHost", 1234);
ResourceManager resourceManager = yarnCluster.getResourceManager();
org.apache.hadoop.yarn.api.records.Token validNMToken =
nmTokenSecretManagerRM.createNMToken(validAppAttemptId, validNode, user);
org.apache.hadoop.yarn.api.records.Token validContainerToken =
containerTokenSecretManager.createContainerToken(validContainerId,
validNode, user, r);
StringBuilder sb;
// testInvalidNMToken ... creating NMToken using different secret manager.
NMTokenSecretManagerInRM tempManager = new NMTokenSecretManagerInRM(conf);
tempManager.rollMasterKey();
do {
tempManager.rollMasterKey();
tempManager.activateNextMasterKey();
// Making sure key id is different.
} while (tempManager.getCurrentKey().getKeyId() == nmTokenSecretManagerRM
.getCurrentKey().getKeyId());
org.apache.hadoop.yarn.api.records.Token invalidNMToken =
tempManager.createNMToken(validAppAttemptId, validNode, user);
sb = new StringBuilder("Given NMToken for application : ");
sb.append(validAppAttemptId.toString())
.append(" seems to have been generated illegally.");
Assert.assertTrue(sb.toString().contains(
testStartContainer(rpc, validAppAttemptId, validNode,
validContainerToken, invalidNMToken, true)));
// valid NMToken but belonging to other node
invalidNMToken =
nmTokenSecretManagerRM.createNMToken(validAppAttemptId, invalidNode,
user);
sb = new StringBuilder("Given NMToken for application : ");
sb.append(validAppAttemptId)
.append(" is not valid for current node manager.expected : ")
.append(validNode.toString())
.append(" found : ").append(invalidNode.toString());
Assert.assertTrue(sb.toString().contains(
testStartContainer(rpc, validAppAttemptId, validNode,
validContainerToken, invalidNMToken, true)));
// using appAttempt-2 token for launching container for appAttempt-1.
invalidNMToken =
nmTokenSecretManagerRM.createNMToken(invalidAppAttemptId, validNode,
user);
sb = new StringBuilder("\nNMToken for application attempt : ");
sb.append(invalidAppAttemptId.toString())
.append(" was used for starting container with container token")
.append(" issued for application attempt : ")
.append(validAppAttemptId.toString());
Assert.assertTrue(testStartContainer(rpc, validAppAttemptId, validNode,
validContainerToken, invalidNMToken, true).contains(sb.toString()));
// using correct tokens. nmtoken for appattempt should get saved.
testStartContainer(rpc, validAppAttemptId, validNode, validContainerToken,
validNMToken, false);
Assert.assertTrue(nmTokenSecretManagerNM
.isAppAttemptNMTokenKeyPresent(validAppAttemptId));
// Rolling over master key twice so that we can check whether older keys
// are used for authentication.
rollNMTokenMasterKey(nmTokenSecretManagerRM, nmTokenSecretManagerNM);
// Key rolled over once.. rolling over again
rollNMTokenMasterKey(nmTokenSecretManagerRM, nmTokenSecretManagerNM);
// trying get container status. Now saved nmToken should be used for
// authentication.
sb = new StringBuilder("Container ");
sb.append(validContainerId.toString());
sb.append(" is not handled by this NodeManager");
Assert.assertTrue(testGetContainer(rpc, validAppAttemptId, validNode,
validContainerId, validNMToken, false).contains(sb.toString()));
}
final YarnRPC yarnRPC = YarnRPC.create(conf); protected void waitForNMToReceiveNMTokenKey(
NMTokenSecretManagerInNM nmTokenSecretManagerNM, NodeManager nm)
throws InterruptedException {
int attempt = 60;
ContainerManagerImpl cm =
((ContainerManagerImpl) nm.getNMContext().getContainerManager());
while ((cm.getBlockNewContainerRequestsStatus() || nmTokenSecretManagerNM
.getNodeId() == null) && attempt-- > 0) {
Thread.sleep(2000);
}
}
// Submit an application protected void rollNMTokenMasterKey(
ApplicationId appID = resourceManager.getClientRMService() NMTokenSecretManagerInRM nmTokenSecretManagerRM,
.getNewApplication(Records.newRecord(GetNewApplicationRequest.class)) NMTokenSecretManagerInNM nmTokenSecretManagerNM) throws Exception {
.getApplicationId(); int oldKeyId = nmTokenSecretManagerRM.getCurrentKey().getKeyId();
ApplicationMasterProtocol scheduler = submitAndRegisterApplication(resourceManager, nmTokenSecretManagerRM.rollMasterKey();
yarnRPC, appID); int interval = 40;
while (nmTokenSecretManagerNM.getCurrentKey().getKeyId() == oldKeyId
&& interval-- > 0) {
Thread.sleep(1000);
}
nmTokenSecretManagerRM.activateNextMasterKey();
Assert.assertTrue((nmTokenSecretManagerNM.getCurrentKey().getKeyId()
== nmTokenSecretManagerRM.getCurrentKey().getKeyId()));
}
// Now request a container. private String testGetContainer(YarnRPC rpc,
final Container allocatedContainer = requestAndGetContainer(scheduler, ApplicationAttemptId appAttemptId, NodeId nodeId,
appID); ContainerId containerId,
org.apache.hadoop.yarn.api.records.Token nmToken,
// Now talk to the NM for launching the container. boolean isExceptionExpected) {
final ContainerId containerID = allocatedContainer.getId(); try {
UserGroupInformation authenticatedUser = UserGroupInformation getContainerStatus(rpc, nmToken, containerId, appAttemptId, nodeId,
.createRemoteUser(containerID.toString()); isExceptionExpected);
org.apache.hadoop.yarn.api.records.Token containerToken = if (isExceptionExpected) {
allocatedContainer.getContainerToken(); fail("Exception was expected!!");
Token<ContainerTokenIdentifier> token = new Token<ContainerTokenIdentifier>(
containerToken.getIdentifier().array(), containerToken.getPassword()
.array(), new Text(containerToken.getKind()), new Text(
containerToken.getService()));
authenticatedUser.addToken(token);
authenticatedUser.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws Exception {
ContainerManagementProtocol client = (ContainerManagementProtocol) yarnRPC.getProxy(
ContainerManagementProtocol.class, NetUtils
.createSocketAddr(allocatedContainer.getNodeId().toString()),
conf);
LOG.info("Going to make a legal stopContainer() request");
StopContainerRequest request = recordFactory
.newRecordInstance(StopContainerRequest.class);
request.setContainerId(containerID);
client.stopContainer(request);
return null;
} }
}); return "";
} catch (Exception e) {
e.printStackTrace();
return e.getMessage();
}
}
KillApplicationRequest request = Records protected String testStartContainer(YarnRPC rpc,
.newRecord(KillApplicationRequest.class); ApplicationAttemptId appAttemptId, NodeId nodeId,
request.setApplicationId(appID); org.apache.hadoop.yarn.api.records.Token containerToken,
resourceManager.getClientRMService().forceKillApplication(request); org.apache.hadoop.yarn.api.records.Token nmToken,
boolean isExceptionExpected) {
try {
startContainer(rpc, nmToken, containerToken, nodeId,
appAttemptId.toString());
if (isExceptionExpected){
fail("Exception was expected!!");
}
return "";
} catch (Exception e) {
e.printStackTrace();
return e.getMessage();
}
}
private void
getContainerStatus(YarnRPC rpc,
org.apache.hadoop.yarn.api.records.Token nmToken,
ContainerId containerId,
ApplicationAttemptId appAttemptId, NodeId nodeId,
boolean isExceptionExpected) throws Exception {
GetContainerStatusRequest request =
Records.newRecord(GetContainerStatusRequest.class);
request.setContainerId(containerId);
ContainerManagementProtocol proxy = null;
try {
proxy =
getContainerManagementProtocolProxy(rpc, nmToken, nodeId,
appAttemptId.toString());
proxy.getContainerStatus(request);
} finally {
if (proxy != null) {
rpc.stopProxy(proxy, conf);
}
}
}
private void startContainer(final YarnRPC rpc,
org.apache.hadoop.yarn.api.records.Token nmToken,
org.apache.hadoop.yarn.api.records.Token containerToken,
NodeId nodeId, String user) throws Exception {
StartContainerRequest request =
Records.newRecord(StartContainerRequest.class);
request.setContainerToken(containerToken);
ContainerLaunchContext context =
Records.newRecord(ContainerLaunchContext.class);
request.setContainerLaunchContext(context);
ContainerManagementProtocol proxy = null;
try {
proxy = getContainerManagementProtocolProxy(rpc, nmToken, nodeId, user);
proxy.startContainer(request);
} finally {
if (proxy != null) {
rpc.stopProxy(proxy, conf);
}
}
}
protected ContainerManagementProtocol getContainerManagementProtocolProxy(
final YarnRPC rpc, org.apache.hadoop.yarn.api.records.Token nmToken,
NodeId nodeId, String user) {
ContainerManagementProtocol proxy;
UserGroupInformation ugi = UserGroupInformation.createRemoteUser(user);
final InetSocketAddress addr =
NetUtils.createSocketAddr(nodeId.getHost(), nodeId.getPort());
ugi.addToken(ConverterUtils.convertFromYarn(nmToken, addr));
proxy = ugi
.doAs(new PrivilegedAction<ContainerManagementProtocol>() {
@Override
public ContainerManagementProtocol run() {
return (ContainerManagementProtocol) rpc.getProxy(
ContainerManagementProtocol.class,
addr, conf);
}
});
return proxy;
} }
/** /**
@ -190,349 +375,61 @@ public class TestContainerManagerSecurity {
* @throws InterruptedException * @throws InterruptedException
* @throws YarnException * @throws YarnException
*/ */
private void testMaliceUser() throws IOException, InterruptedException, private void testContainerToken(Configuration conf) throws IOException,
YarnException { InterruptedException, YarnException {
LOG.info("Running test for malice user"); LOG.info("Running test for malice user");
/*
ResourceManager resourceManager = yarnCluster.getResourceManager(); * We need to check for containerToken (authorization).
* Here we will be assuming that we have valid NMToken
final YarnRPC yarnRPC = YarnRPC.create(conf); * 1) ContainerToken used is expired.
* 2) ContainerToken is tampered (resource is modified).
// Submit an application */
ApplicationId appID = resourceManager.getClientRMService() NMTokenSecretManagerInRM nmTokenSecretManagerInRM =
.getNewApplication(Records.newRecord(GetNewApplicationRequest.class)) yarnCluster.getResourceManager().getRMContext()
.getApplicationId(); .getNMTokenSecretManager();
ApplicationMasterProtocol scheduler = submitAndRegisterApplication(resourceManager, ApplicationId appId = ApplicationId.newInstance(1, 1);
yarnRPC, appID); ApplicationAttemptId appAttemptId =
ApplicationAttemptId.newInstance(appId, 0);
// Now request a container. ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
final Container allocatedContainer = requestAndGetContainer(scheduler, NodeManager nm = yarnCluster.getNodeManager(0);
appID); NMTokenSecretManagerInNM nmTokenSecretManagerInNM =
nm.getNMContext().getNMTokenSecretManager();
// Now talk to the NM for launching the container with modified resource String user = "test";
org.apache.hadoop.yarn.api.records.Token containerToken =
allocatedContainer.getContainerToken();
ContainerTokenIdentifier originalContainerTokenId =
BuilderUtils.newContainerTokenIdentifier(containerToken);
// Malice user modifies the resource amount
Resource modifiedResource = BuilderUtils.newResource(2048, 1);
ContainerTokenIdentifier modifiedIdentifier =
new ContainerTokenIdentifier(originalContainerTokenId.getContainerID(),
originalContainerTokenId.getNmHostAddress(), "testUser",
modifiedResource, Long.MAX_VALUE,
originalContainerTokenId.getMasterKeyId(),
ResourceManager.clusterTimeStamp);
Token<ContainerTokenIdentifier> modifiedToken =
new Token<ContainerTokenIdentifier>(modifiedIdentifier.getBytes(),
containerToken.getPassword().array(), new Text(
containerToken.getKind()), new Text(containerToken.getService()));
makeTamperedStartContainerCall(yarnRPC, allocatedContainer,
modifiedIdentifier, modifiedToken);
// Malice user modifies the container-Id
ContainerId newContainerId =
BuilderUtils.newContainerId(
BuilderUtils.newApplicationAttemptId(originalContainerTokenId
.getContainerID().getApplicationAttemptId().getApplicationId(), 1),
originalContainerTokenId.getContainerID().getId() + 42);
modifiedIdentifier =
new ContainerTokenIdentifier(newContainerId,
originalContainerTokenId.getNmHostAddress(), "testUser",
originalContainerTokenId.getResource(), Long.MAX_VALUE,
originalContainerTokenId.getMasterKeyId(),
ResourceManager.clusterTimeStamp);
modifiedToken =
new Token<ContainerTokenIdentifier>(modifiedIdentifier.getBytes(),
containerToken.getPassword().array(), new Text(
containerToken.getKind()), new Text(containerToken.getService()));
makeTamperedStartContainerCall(yarnRPC, allocatedContainer,
modifiedIdentifier, modifiedToken);
// Similarly messing with anything else will fail.
KillApplicationRequest request = Records
.newRecord(KillApplicationRequest.class);
request.setApplicationId(appID);
resourceManager.getClientRMService().forceKillApplication(request);
}
private void makeTamperedStartContainerCall(final YarnRPC yarnRPC,
final Container allocatedContainer,
final ContainerTokenIdentifier modifiedIdentifier,
Token<ContainerTokenIdentifier> modifiedToken) {
final ContainerId containerID = allocatedContainer.getId();
UserGroupInformation maliceUser = UserGroupInformation
.createRemoteUser(containerID.toString());
maliceUser.addToken(modifiedToken);
maliceUser.doAs(new PrivilegedAction<Void>() {
@Override
public Void run() {
ContainerManagementProtocol client = (ContainerManagementProtocol) yarnRPC.getProxy(
ContainerManagementProtocol.class, NetUtils
.createSocketAddr(allocatedContainer.getNodeId().toString()),
conf);
LOG.info("Going to contact NM: ilLegal request");
StartContainerRequest request =
Records.newRecord(StartContainerRequest.class);
try {
request.setContainerToken(allocatedContainer.getContainerToken());
ContainerLaunchContext context =
createContainerLaunchContextForTest(modifiedIdentifier);
request.setContainerLaunchContext(context);
client.startContainer(request);
fail("Connection initiation with illegally modified "
+ "tokens is expected to fail.");
} catch (YarnException e) {
LOG.error("Got exception", e);
fail("Cannot get a YARN remote exception as "
+ "it will indicate RPC success");
} catch (Exception e) {
Assert.assertEquals(
javax.security.sasl.SaslException.class
.getCanonicalName(), e.getClass().getCanonicalName());
Assert.assertTrue(e
.getMessage()
.contains(
"DIGEST-MD5: digest response format violation. "
+ "Mismatched response."));
}
return null;
}
});
}
private void testExpiredTokens() throws IOException, InterruptedException,
YarnException {
LOG.info("\n\nRunning test for malice user");
ResourceManager resourceManager = yarnCluster.getResourceManager();
final YarnRPC yarnRPC = YarnRPC.create(conf);
// Submit an application
final ApplicationId appID = resourceManager.getClientRMService()
.getNewApplication(Records.newRecord(GetNewApplicationRequest.class))
.getApplicationId();
ApplicationMasterProtocol scheduler = submitAndRegisterApplication(resourceManager,
yarnRPC, appID);
// Now request a container.
final Container allocatedContainer = requestAndGetContainer(scheduler,
appID);
// Now talk to the NM for launching the container with modified containerID
final ContainerId containerID = allocatedContainer.getId();
org.apache.hadoop.yarn.api.records.Token containerToken =
allocatedContainer.getContainerToken();
final ContainerTokenIdentifier tokenId =
BuilderUtils.newContainerTokenIdentifier(containerToken);
/////////// Test calls with expired tokens
UserGroupInformation unauthorizedUser = UserGroupInformation
.createRemoteUser(containerID.toString());
RMContainerTokenSecretManager containerTokenSecreteManager =
resourceManager.getRMContainerTokenSecretManager();
final ContainerTokenIdentifier newTokenId =
new ContainerTokenIdentifier(tokenId.getContainerID(),
tokenId.getNmHostAddress(), tokenId.getApplicationSubmitter(),
tokenId.getResource(), System.currentTimeMillis() - 1,
containerTokenSecreteManager.getCurrentKey().getKeyId(),
ResourceManager.clusterTimeStamp);
final byte[] passowrd =
containerTokenSecreteManager.createPassword(
newTokenId);
// Create a valid token by using the key from the RM.
Token<ContainerTokenIdentifier> token =
new Token<ContainerTokenIdentifier>(newTokenId.getBytes(), passowrd,
new Text(containerToken.getKind()), new Text(
containerToken.getService()));
unauthorizedUser.addToken(token);
unauthorizedUser.doAs(new PrivilegedAction<Void>() {
@Override
public Void run() {
ContainerManagementProtocol client = (ContainerManagementProtocol) yarnRPC.getProxy(
ContainerManagementProtocol.class, NetUtils
.createSocketAddr(allocatedContainer.getNodeId().toString()),
conf);
LOG.info("Going to contact NM with expired token");
ContainerLaunchContext context = createContainerLaunchContextForTest(newTokenId);
StartContainerRequest request =
Records.newRecord(StartContainerRequest.class);
request.setContainerLaunchContext(context);
allocatedContainer.setContainerToken(BuilderUtils.newContainerToken(
allocatedContainer.getNodeId(), passowrd, newTokenId));
request.setContainerToken(allocatedContainer.getContainerToken());
//Calling startContainer with an expired token.
try {
client.startContainer(request);
fail("Connection initiation with expired "
+ "token is expected to fail.");
} catch (Throwable t) {
LOG.info("Got exception : ", t);
Assert.assertTrue(t.getMessage().contains(
"This token is expired. current time is"));
}
// Try stopping a container - should not get an expiry error.
StopContainerRequest stopRequest = Records.newRecord(StopContainerRequest.class);
stopRequest.setContainerId(newTokenId.getContainerID());
try {
client.stopContainer(stopRequest);
} catch (Throwable t) {
fail("Stop Container call should have succeeded");
}
return null;
}
});
/////////// End of testing calls with expired tokens
KillApplicationRequest request = Records
.newRecord(KillApplicationRequest.class);
request.setApplicationId(appID);
resourceManager.getClientRMService().forceKillApplication(request);
}
private ApplicationMasterProtocol submitAndRegisterApplication(
ResourceManager resourceManager, final YarnRPC yarnRPC,
ApplicationId appID) throws IOException,
UnsupportedFileSystemException, YarnException,
InterruptedException {
// Use ping to simulate sleep on Windows.
List<String> cmd = Shell.WINDOWS ?
Arrays.asList("ping", "-n", "100", "127.0.0.1", ">nul") :
Arrays.asList("sleep", "100");
ContainerLaunchContext amContainer =
BuilderUtils.newContainerLaunchContext(
Collections.<String, LocalResource> emptyMap(),
new HashMap<String, String>(), cmd,
new HashMap<String, ByteBuffer>(), null,
new HashMap<ApplicationAccessType, String>());
ApplicationSubmissionContext appSubmissionContext = recordFactory
.newRecordInstance(ApplicationSubmissionContext.class);
appSubmissionContext.setApplicationId(appID);
appSubmissionContext.setAMContainerSpec(amContainer);
appSubmissionContext.setResource(BuilderUtils.newResource(1024, 1));
SubmitApplicationRequest submitRequest = recordFactory
.newRecordInstance(SubmitApplicationRequest.class);
submitRequest.setApplicationSubmissionContext(appSubmissionContext);
resourceManager.getClientRMService().submitApplication(submitRequest);
// Wait till container gets allocated for AM
int waitCounter = 0;
RMApp app = resourceManager.getRMContext().getRMApps().get(appID);
RMAppAttempt appAttempt = app == null ? null : app.getCurrentAppAttempt();
RMAppAttemptState state = appAttempt == null ? null : appAttempt
.getAppAttemptState();
while ((app == null || appAttempt == null || state == null || !state
.equals(RMAppAttemptState.LAUNCHED))
&& waitCounter++ != 20) {
LOG.info("Waiting for applicationAttempt to be created.. ");
Thread.sleep(1000);
app = resourceManager.getRMContext().getRMApps().get(appID);
appAttempt = app == null ? null : app.getCurrentAppAttempt();
state = appAttempt == null ? null : appAttempt.getAppAttemptState();
}
Assert.assertNotNull(app);
Assert.assertNotNull(appAttempt);
Assert.assertNotNull(state);
Assert.assertEquals(RMAppAttemptState.LAUNCHED, state);
UserGroupInformation currentUser = UserGroupInformation.createRemoteUser(
appAttempt.getAppAttemptId().toString());
// Ask for a container from the RM
final InetSocketAddress schedulerAddr =
resourceManager.getApplicationMasterService().getBindAddress();
if (UserGroupInformation.isSecurityEnabled()) {
AMRMTokenIdentifier appTokenIdentifier = new AMRMTokenIdentifier(
appAttempt.getAppAttemptId());
AMRMTokenSecretManager appTokenSecretManager =
new AMRMTokenSecretManager(conf);
appTokenSecretManager.setMasterKey(resourceManager
.getAMRMTokenSecretManager().getMasterKey());
Token<AMRMTokenIdentifier> appToken =
new Token<AMRMTokenIdentifier>(appTokenIdentifier,
appTokenSecretManager);
SecurityUtil.setTokenService(appToken, schedulerAddr);
currentUser.addToken(appToken);
}
ApplicationMasterProtocol scheduler = currentUser waitForNMToReceiveNMTokenKey(nmTokenSecretManagerInNM, nm);
.doAs(new PrivilegedAction<ApplicationMasterProtocol>() {
@Override
public ApplicationMasterProtocol run() {
return (ApplicationMasterProtocol) yarnRPC.getProxy(ApplicationMasterProtocol.class,
schedulerAddr, conf);
}
});
// Register the appMaster NodeId nodeId = nm.getNMContext().getNodeId();
RegisterApplicationMasterRequest request = recordFactory
.newRecordInstance(RegisterApplicationMasterRequest.class); // Both id should be equal.
request.setApplicationAttemptId(resourceManager.getRMContext() Assert.assertEquals(nmTokenSecretManagerInNM.getCurrentKey().getKeyId(),
.getRMApps().get(appID).getCurrentAppAttempt().getAppAttemptId()); nmTokenSecretManagerInRM.getCurrentKey().getKeyId());
scheduler.registerApplicationMaster(request);
return scheduler; // Creating a tampered Container Token
} RMContainerTokenSecretManager containerTokenSecretManager =
yarnCluster.getResourceManager().getRMContainerTokenSecretManager();
private Container requestAndGetContainer(ApplicationMasterProtocol scheduler,
ApplicationId appID) throws YarnException, InterruptedException, RMContainerTokenSecretManager tamperedContainerTokenSecretManager =
IOException { new RMContainerTokenSecretManager(conf);
tamperedContainerTokenSecretManager.rollMasterKey();
// Request a container allocation. do {
List<ResourceRequest> ask = new ArrayList<ResourceRequest>(); tamperedContainerTokenSecretManager.rollMasterKey();
ask.add(BuilderUtils.newResourceRequest(BuilderUtils.newPriority(0), tamperedContainerTokenSecretManager.activateNextMasterKey();
ResourceRequest.ANY, BuilderUtils.newResource(1024, 1), 1)); } while (containerTokenSecretManager.getCurrentKey().getKeyId()
== tamperedContainerTokenSecretManager.getCurrentKey().getKeyId());
AllocateRequest allocateRequest = AllocateRequest.newInstance(
BuilderUtils.newApplicationAttemptId(appID, 1), 0, 0F, ask, Resource r = Resource.newInstance(1230, 2);
new ArrayList<ContainerId>(), null); // Creating modified containerToken
List<Container> allocatedContainers = scheduler.allocate(allocateRequest) Token containerToken =
.getAllocatedContainers(); tamperedContainerTokenSecretManager.createContainerToken(cId, nodeId,
user, r);
// Modify ask to request no more. Token nmToken =
allocateRequest.setAskList(new ArrayList<ResourceRequest>()); nmTokenSecretManagerInRM.createNMToken(appAttemptId, nodeId, user);
YarnRPC rpc = YarnRPC.create(conf);
int waitCounter = 0; StringBuilder sb = new StringBuilder("Given Container ");
while ((allocatedContainers == null || allocatedContainers.size() == 0) sb.append(cId);
&& waitCounter++ != 20) { sb.append(" seems to have an illegally generated token.");
LOG.info("Waiting for container to be allocated.."); Assert.assertTrue(testStartContainer(rpc, appAttemptId, nodeId,
Thread.sleep(1000); containerToken, nmToken, true).contains(sb.toString()));
allocateRequest.setResponseId(allocateRequest.getResponseId() + 1);
allocatedContainers = scheduler.allocate(allocateRequest)
.getAllocatedContainers();
}
Assert.assertNotNull("Container is not allocted!", allocatedContainers);
Assert.assertEquals("Didn't get one container!", 1, allocatedContainers
.size());
return allocatedContainers.get(0);
}
private ContainerLaunchContext createContainerLaunchContextForTest(
ContainerTokenIdentifier tokenId) {
ContainerLaunchContext context =
BuilderUtils.newContainerLaunchContext(
new HashMap<String, LocalResource>(),
new HashMap<String, String>(), new ArrayList<String>(),
new HashMap<String, ByteBuffer>(), null,
new HashMap<ApplicationAccessType, String>());
return context;
} }
} }