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:
parent
8e81e6fcaf
commit
368c7ae735
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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")
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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.
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
|
@ -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());
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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;
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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",
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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;
|
||||||
}
|
}
|
||||||
};
|
};
|
||||||
|
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -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
|
||||||
|
|
|
@ -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>
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -435,7 +435,7 @@ public class NodeManager extends CompositeService
|
||||||
}
|
}
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
Context getNMContext() {
|
public Context getNMContext() {
|
||||||
return this.context;
|
return this.context;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -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);
|
||||||
|
}
|
||||||
}
|
}
|
|
@ -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;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
|
||||||
}
|
|
||||||
}
|
}
|
|
@ -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()) {
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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
|
||||||
|
|
|
@ -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);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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>() {
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -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;
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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,
|
||||||
|
|
|
@ -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);
|
||||||
|
|
|
@ -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>
|
|
@ -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;
|
||||||
|
|
|
@ -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:
|
||||||
|
|
|
@ -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);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -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) {
|
||||||
|
|
|
@ -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;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue