MAPREDUCE-3940. ContainerTokens should have an expiry interval. Contributed by Siddharth Seth and Vinod Kumar Vavilapalli.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1359910 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2012-07-10 21:26:48 +00:00
parent 3b5ea87502
commit 3bfb26ad3b
18 changed files with 217 additions and 56 deletions

View File

@ -670,6 +670,9 @@ Release 0.23.3 - UNRELEASED
MAPREDUCE-4252. MR2 job never completes with 1 pending task (Tom White via
bobby)
MAPREDUCE-3940. ContainerTokens should have an expiry interval. (Siddharth
Seth and Vinod Kumar Vavilapalli via vinodkv)
Release 0.23.2 - UNRELEASED
INCOMPATIBLE CHANGES

View File

@ -29,6 +29,7 @@
import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
@ -86,11 +87,13 @@
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo.FifoScheduler;
import org.apache.hadoop.yarn.server.security.ContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils;
import org.junit.After;
import org.junit.Test;
@ -352,7 +355,7 @@ public void handle(SchedulerEvent event) {
}
@Override
protected ResourceScheduler createScheduler() {
return new MyFifoScheduler();
return new MyFifoScheduler(this.getRMContext());
}
}
@ -1091,6 +1094,19 @@ public void testBlackListedNodesWithSchedulingToThatNode() throws Exception {
}
private static class MyFifoScheduler extends FifoScheduler {
public MyFifoScheduler(RMContext rmContext) {
super();
try {
Configuration conf = new Configuration();
reinitialize(conf, new ContainerTokenSecretManager(conf),
rmContext);
} catch (IOException ie) {
LOG.info("add application failed with ", ie);
assert (false);
}
}
// override this to copy the objects otherwise FifoScheduler updates the
// numContainers in same objects as kept by RMContainerAllocator
@Override

View File

@ -50,11 +50,11 @@ public interface ContainerToken extends DelegationToken {
*/
@Public
@Stable
public abstract ByteBuffer getIdentifier();
ByteBuffer getIdentifier();
@Private
@Stable
public abstract void setIdentifier(ByteBuffer identifier);
void setIdentifier(ByteBuffer identifier);
/**
* Get the token password
@ -62,11 +62,11 @@ public interface ContainerToken extends DelegationToken {
*/
@Public
@Stable
public abstract ByteBuffer getPassword();
ByteBuffer getPassword();
@Private
@Stable
public abstract void setPassword(ByteBuffer password);
void setPassword(ByteBuffer password);
/**
* Get the token kind.
@ -74,11 +74,11 @@ public interface ContainerToken extends DelegationToken {
*/
@Public
@Stable
public abstract String getKind();
String getKind();
@Private
@Stable
public abstract void setKind(String kind);
void setKind(String kind);
/**
* Get the service to which the token is allocated.
@ -86,10 +86,10 @@ public interface ContainerToken extends DelegationToken {
*/
@Public
@Stable
public abstract String getService();
String getService();
@Private
@Stable
public abstract void setService(String service);
void setService(String service);
}

View File

@ -20,6 +20,7 @@
import java.io.IOException;
import java.net.InetSocketAddress;
import java.io.Closeable;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.ipc.ProtobufRpcEngine;
@ -49,7 +50,8 @@
import com.google.protobuf.ServiceException;
public class ContainerManagerPBClientImpl implements ContainerManager {
public class ContainerManagerPBClientImpl implements ContainerManager,
Closeable {
// Not a documented config. Only used for tests
static final String NM_COMMAND_TIMEOUT = YarnConfiguration.YARN_PREFIX

View File

@ -35,6 +35,11 @@
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.util.BuilderUtils;
/**
* TokenIdentifier for a container. Encodes {@link ContainerId},
* {@link Resource} needed by the container and the target NMs host-address.
*
*/
public class ContainerTokenIdentifier extends TokenIdentifier {
private static Log LOG = LogFactory.getLog(ContainerTokenIdentifier.class);
@ -44,14 +49,19 @@ public class ContainerTokenIdentifier extends TokenIdentifier {
private ContainerId containerId;
private String nmHostAddr;
private Resource resource;
private long expiryTimeStamp;
public ContainerTokenIdentifier(ContainerId containerID, String hostName,
Resource r) {
Resource r, long expiryTimeStamp) {
this.containerId = containerID;
this.nmHostAddr = hostName;
this.resource = r;
this.expiryTimeStamp = expiryTimeStamp;
}
/**
* Default constructor needed by RPC layer/SecretManager.
*/
public ContainerTokenIdentifier() {
}
@ -67,6 +77,10 @@ public Resource getResource() {
return this.resource;
}
public long getExpiryTimeStamp() {
return this.expiryTimeStamp;
}
@Override
public void write(DataOutput out) throws IOException {
LOG.debug("Writing ContainerTokenIdentifier to RPC layer: " + this);
@ -79,6 +93,7 @@ public void write(DataOutput out) throws IOException {
out.writeInt(this.containerId.getId());
out.writeUTF(this.nmHostAddr);
out.writeInt(this.resource.getMemory());
out.writeLong(this.expiryTimeStamp);
}
@Override
@ -91,6 +106,7 @@ public void readFields(DataInput in) throws IOException {
.readInt());
this.nmHostAddr = in.readUTF();
this.resource = BuilderUtils.newResource(in.readInt());
this.expiryTimeStamp = in.readLong();
}
@Override
@ -103,6 +119,7 @@ public UserGroupInformation getUser() {
return UserGroupInformation.createRemoteUser(this.containerId.toString());
}
// TODO: Needed?
@InterfaceAudience.Private
public static class Renewer extends Token.TrivialRenewer {
@Override

View File

@ -18,6 +18,7 @@
package org.apache.hadoop.yarn.server.security;
import java.nio.ByteBuffer;
import java.util.Map;
import java.util.concurrent.ConcurrentHashMap;
@ -25,9 +26,21 @@
import org.apache.commons.logging.Log;
import org.apache.commons.logging.LogFactory;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.token.SecretManager;
import org.apache.hadoop.yarn.api.records.ContainerId;
import org.apache.hadoop.yarn.api.records.ContainerToken;
import org.apache.hadoop.yarn.api.records.NodeId;
import org.apache.hadoop.yarn.api.records.Resource;
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.util.BuilderUtils;
/**
* SecretManager for ContainerTokens. Used by both RM and NM and hence is
* present in yarn-server-common package.
*
*/
public class ContainerTokenSecretManager extends
SecretManager<ContainerTokenIdentifier> {
@ -36,7 +49,34 @@ public class ContainerTokenSecretManager extends
Map<String, SecretKey> secretkeys =
new ConcurrentHashMap<String, SecretKey>();
private final long containerTokenExpiryInterval;
public ContainerTokenSecretManager(Configuration conf) {
this.containerTokenExpiryInterval =
conf.getInt(YarnConfiguration.RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS,
YarnConfiguration.DEFAULT_RM_CONTAINER_ALLOC_EXPIRY_INTERVAL_MS);
}
public ContainerToken createContainerToken(ContainerId containerId,
NodeId nodeId, Resource capability) {
try {
long expiryTimeStamp =
System.currentTimeMillis() + containerTokenExpiryInterval;
ContainerTokenIdentifier tokenIdentifier =
new ContainerTokenIdentifier(containerId, nodeId.toString(),
capability, expiryTimeStamp);
return BuilderUtils.newContainerToken(nodeId,
ByteBuffer.wrap(this.createPassword(tokenIdentifier)), tokenIdentifier);
} catch (IllegalArgumentException e) {
// this could be because DNS is down - in which case we just want
// to retry and not bring RM down. Caller should note and act on the fact
// that container is not creatable.
LOG.error("Error trying to create new container", e);
return null;
}
}
// Used by master for generation of secretyKey per host
public SecretKey createAndGetSecretKey(CharSequence hostName) {
String hostNameStr = hostName.toString();

View File

@ -116,7 +116,7 @@ public void init(Configuration conf) {
if (UserGroupInformation.isSecurityEnabled()) {
LOG.info("Security is enabled on NodeManager. "
+ "Creating ContainerTokenSecretManager");
this.containerTokenSecretManager = new ContainerTokenSecretManager();
this.containerTokenSecretManager = new ContainerTokenSecretManager(conf);
}
this.aclsManager = new ApplicationACLsManager(conf);

View File

@ -324,6 +324,15 @@ private void authorizeRequest(ContainerId containerID,
+ containerIDStr);
} else {
// Ensure the token is not expired.
// Token expiry is not checked for stopContainer/getContainerStatus
if (tokenId.getExpiryTimeStamp() < System.currentTimeMillis()) {
unauthorized = true;
messageBuilder.append("\nThis token is expired. current time is "
+ System.currentTimeMillis() + " found "
+ tokenId.getExpiryTimeStamp());
}
Resource resource = tokenId.getResource();
if (!resource.equals(launchContext.getResource())) {
unauthorized = true;

View File

@ -86,7 +86,8 @@ public void testSuccessfulContainerLaunch() throws InterruptedException,
healthChecker.init(conf);
LocalDirsHandlerService dirsHandler = healthChecker.getDiskHandler();
NodeManagerMetrics metrics = NodeManagerMetrics.create();
ContainerTokenSecretManager containerTokenSecretManager = new ContainerTokenSecretManager();
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(conf);
NodeStatusUpdater nodeStatusUpdater =
new NodeStatusUpdaterImpl(context, dispatcher, healthChecker, metrics, containerTokenSecretManager) {
@Override

View File

@ -70,7 +70,8 @@ public abstract class BaseContainerManagerTest {
protected static File localLogDir;
protected static File remoteLogDir;
protected static File tmpDir;
protected ContainerTokenSecretManager containerTokenSecretManager = new ContainerTokenSecretManager();
protected ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(new Configuration());
protected final NodeManagerMetrics metrics = NodeManagerMetrics.create();

View File

@ -385,7 +385,7 @@ public void testLocalFilesCleanup() throws InterruptedException,
delSrvc.init(conf);
ContainerTokenSecretManager containerTokenSecretManager = new
ContainerTokenSecretManager();
ContainerTokenSecretManager(conf);
containerManager = new ContainerManagerImpl(context, exec, delSrvc,
nodeStatusUpdater, metrics, containerTokenSecretManager,
new ApplicationACLsManager(conf), dirsHandler);

View File

@ -100,8 +100,7 @@ public class ResourceManager extends CompositeService implements Recoverable {
protected ClientToAMSecretManager clientToAMSecretManager =
new ClientToAMSecretManager();
protected ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager();
protected ContainerTokenSecretManager containerTokenSecretManager;
protected ApplicationTokenSecretManager appTokenSecretManager;
@ -151,6 +150,8 @@ public synchronized void init(Configuration conf) {
this.rmDispatcher);
addService(this.containerAllocationExpirer);
this.containerTokenSecretManager = new ContainerTokenSecretManager(conf);
AMLivelinessMonitor amLivelinessMonitor = createAMLivelinessMonitor();
addService(amLivelinessMonitor);
@ -611,6 +612,11 @@ public ApplicationACLsManager getApplicationACLsManager() {
return this.applicationACLsManager;
}
@Private
public ContainerTokenSecretManager getContainerTokenSecretManager() {
return this.containerTokenSecretManager;
}
@Private
public ApplicationTokenSecretManager getApplicationTokenSecretManager(){
return this.appTokenSecretManager;

View File

@ -27,6 +27,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.ContainerExpiredSchedulerEvent;
import org.apache.hadoop.yarn.util.AbstractLivelinessMonitor;
@SuppressWarnings({"unchecked", "rawtypes"})
public class ContainerAllocationExpirer extends
AbstractLivelinessMonitor<ContainerId> {

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
@ -54,7 +53,6 @@
import org.apache.hadoop.yarn.api.records.ResourceRequest;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.rmcontainer.RMContainerEventType;
@ -1178,17 +1176,11 @@ public Container createContainer(SchedulerApp application, SchedulerNode node,
// If security is enabled, send the container-tokens too.
if (UserGroupInformation.isSecurityEnabled()) {
ContainerTokenIdentifier tokenIdentifier = new ContainerTokenIdentifier(
containerId, nodeId.toString(), capability);
try {
containerToken = BuilderUtils.newContainerToken(nodeId, ByteBuffer
.wrap(containerTokenSecretManager
.createPassword(tokenIdentifier)), tokenIdentifier);
} catch (IllegalArgumentException e) {
// this could be because DNS is down - in which case we just want
// to retry and not bring RM down
LOG.error("Error trying to create new container", e);
return null;
containerToken =
containerTokenSecretManager.createContainerToken(containerId, nodeId,
capability);
if (containerToken == null) {
return null; // Try again later.
}
}

View File

@ -19,7 +19,6 @@
package org.apache.hadoop.yarn.server.resourcemanager.scheduler.fifo;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collections;
@ -32,7 +31,6 @@
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.classification.InterfaceStability.Evolving;
import org.apache.hadoop.conf.Configurable;
import org.apache.hadoop.conf.Configuration;
@ -55,7 +53,6 @@
import org.apache.hadoop.yarn.conf.YarnConfiguration;
import org.apache.hadoop.yarn.factories.RecordFactory;
import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger;
import org.apache.hadoop.yarn.server.resourcemanager.RMAuditLogger.AuditConstants;
import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@ -541,11 +538,12 @@ private int assignContainer(SchedulerNode node, SchedulerApp application,
// If security is enabled, send the container-tokens too.
if (UserGroupInformation.isSecurityEnabled()) {
ContainerTokenIdentifier tokenIdentifier = new ContainerTokenIdentifier(
containerId, nodeId.toString(), capability);
containerToken = BuilderUtils.newContainerToken(nodeId, ByteBuffer
.wrap(containerTokenSecretManager
.createPassword(tokenIdentifier)), tokenIdentifier);
containerToken =
containerTokenSecretManager.createContainerToken(containerId,
nodeId, capability);
if (containerToken == null) {
return i; // Try again later.
}
}
// Create the container
@ -562,11 +560,11 @@ private int assignContainer(SchedulerNode node, SchedulerApp application,
// Inform the node
node.allocateContainer(application.getApplicationId(),
rmContainer);
// Update usage for this container
Resources.addTo(usedResource, capability);
}
// Update total usage
Resources.addTo(usedResource,
Resources.multiply(capability, assignedContainers));
}
return assignedContainers;

View File

@ -52,8 +52,6 @@ public class TestNMExpiry {
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
ResourceTrackerService resourceTrackerService;
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager();
private class TestNmLivelinessMonitor extends NMLivelinessMonitor {
public TestNmLivelinessMonitor(Dispatcher dispatcher) {
@ -84,6 +82,8 @@ public void setUp() {
nmLivelinessMonitor.start();
NodesListManager nodesListManager = new NodesListManager(context);
nodesListManager.init(conf);
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(conf);
resourceTrackerService = new ResourceTrackerService(context,
nodesListManager, nmLivelinessMonitor, containerTokenSecretManager);

View File

@ -52,8 +52,6 @@
public class TestRMNMRPCResponseId {
private static final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
ResourceTrackerService resourceTrackerService;
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager();
private NodeId nodeId;
@Before
@ -73,6 +71,8 @@ public void handle(Event event) {
NodesListManager nodesListManager = new NodesListManager(context);
Configuration conf = new Configuration();
nodesListManager.init(conf);
ContainerTokenSecretManager containerTokenSecretManager =
new ContainerTokenSecretManager(conf);
resourceTrackerService = new ResourceTrackerService(context,
nodesListManager, new NMLivelinessMonitor(dispatcher),
containerTokenSecretManager);

View File

@ -45,6 +45,7 @@
import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SecurityUtil;
@ -214,11 +215,12 @@ public void testMaliceUser() throws IOException, InterruptedException {
ContainerTokenIdentifier dummyIdentifier = new ContainerTokenIdentifier();
dummyIdentifier.readFields(di);
// Malice user modifies the resource amount
Resource modifiedResource = BuilderUtils.newResource(2048);
ContainerTokenIdentifier modifiedIdentifier = new ContainerTokenIdentifier(
dummyIdentifier.getContainerID(), dummyIdentifier.getNmHostAddress(),
modifiedResource);
modifiedResource, Long.MAX_VALUE);
Token<ContainerTokenIdentifier> modifiedToken = new Token<ContainerTokenIdentifier>(
modifiedIdentifier.getBytes(), containerToken.getPassword().array(),
new Text(containerToken.getKind()), new Text(containerToken
@ -288,6 +290,7 @@ public void testUnauthorizedUser() throws IOException, InterruptedException {
// Now talk to the NM for launching the container with modified containerID
final ContainerId containerID = allocatedContainer.getId();
/////////// Test calls with illegal containerIDs and illegal Resources
UserGroupInformation unauthorizedUser = UserGroupInformation
.createRemoteUser(containerID.toString());
ContainerToken containerToken = allocatedContainer.getContainerToken();
@ -303,9 +306,10 @@ public void testUnauthorizedUser() throws IOException, InterruptedException {
containerToken.getKind()), new Text(containerToken.getService()));
unauthorizedUser.addToken(token);
unauthorizedUser.doAs(new PrivilegedAction<Void>() {
ContainerManager client =
unauthorizedUser.doAs(new PrivilegedAction<ContainerManager>() {
@Override
public Void run() {
public ContainerManager run() {
ContainerManager client = (ContainerManager) yarnRPC.getProxy(
ContainerManager.class, NetUtils
.createSocketAddr(allocatedContainer.getNodeId().toString()),
@ -316,16 +320,76 @@ public Void run() {
callWithIllegalContainerID(client, tokenId);
callWithIllegalResource(client, tokenId);
return client;
}
});
/////////// End of testing for illegal containerIDs and illegal Resources
/////////// Test calls with expired tokens
RPC.stopProxy(client);
unauthorizedUser = UserGroupInformation
.createRemoteUser(containerID.toString());
final ContainerTokenIdentifier newTokenId =
new ContainerTokenIdentifier(tokenId.getContainerID(),
tokenId.getNmHostAddress(), tokenId.getResource(),
System.currentTimeMillis() - 1);
byte[] passowrd =
resourceManager.getContainerTokenSecretManager().createPassword(
newTokenId);
// Create a valid token by using the key from the RM.
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() {
ContainerManager client = (ContainerManager) yarnRPC.getProxy(
ContainerManager.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);
//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 AMRMProtocol submitAndRegisterApplication(
ResourceManager resourceManager, final YarnRPC yarnRPC,
ApplicationId appID) throws IOException,
@ -481,11 +545,9 @@ void callWithIllegalResource(ContainerManager client,
StartContainerRequest request = recordFactory
.newRecordInstance(StartContainerRequest.class);
// Authenticated but unauthorized, due to wrong resource
ContainerLaunchContext context = BuilderUtils.newContainerLaunchContext(
tokenId.getContainerID(), "testUser", BuilderUtils.newResource(2048),
new HashMap<String, LocalResource>(), new HashMap<String, String>(),
new ArrayList<String>(), new HashMap<String, ByteBuffer>(), null,
new HashMap<ApplicationAccessType, String>());
ContainerLaunchContext context =
createContainerLaunchContextForTest(tokenId);
context.getResource().setMemory(2048); // Set a different resource size.
request.setContainerLaunchContext(context);
try {
client.startContainer(request);
@ -500,4 +562,17 @@ void callWithIllegalResource(ContainerManager client,
+ " but found " + context.getResource().toString()));
}
}
private ContainerLaunchContext createContainerLaunchContextForTest(
ContainerTokenIdentifier tokenId) {
ContainerLaunchContext context =
BuilderUtils.newContainerLaunchContext(tokenId.getContainerID(),
"testUser",
BuilderUtils.newResource(tokenId.getResource().getMemory()),
new HashMap<String, LocalResource>(),
new HashMap<String, String>(), new ArrayList<String>(),
new HashMap<String, ByteBuffer>(), null,
new HashMap<ApplicationAccessType, String>());
return context;
}
}