YARN-617. Made ContainerTokens to be used for validation at NodeManager also in unsecure mode to prevent AMs from faking resource requirements in unsecure mode. Contributed by Omkar Vinit Joshi.

Merge from trunk revision 1483667 with little test modifications for proper merge.


git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1483668 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Vinod Kumar Vavilapalli 2013-05-17 06:38:03 +00:00
parent 5fc2beed58
commit f0bd8da38b
39 changed files with 747 additions and 447 deletions

View File

@ -350,15 +350,14 @@ protected ContainerManager getCMProxy(ContainerId containerID,
final InetSocketAddress cmAddr = final InetSocketAddress cmAddr =
NetUtils.createSocketAddr(containerManagerBindAddr); NetUtils.createSocketAddr(containerManagerBindAddr);
UserGroupInformation user = UserGroupInformation.getCurrentUser();
if (UserGroupInformation.isSecurityEnabled()) { // the user in createRemoteUser in this context has to be ContainerID
UserGroupInformation user =
UserGroupInformation.createRemoteUser(containerID.toString());
Token<ContainerTokenIdentifier> token = Token<ContainerTokenIdentifier> token =
ProtoUtils.convertFromProtoFormat(containerToken, cmAddr); ProtoUtils.convertFromProtoFormat(containerToken, cmAddr);
// the user in createRemoteUser in this context has to be ContainerID
user = UserGroupInformation.createRemoteUser(containerID.toString());
user.addToken(token); user.addToken(token);
}
ContainerManager proxy = user ContainerManager proxy = user
.doAs(new PrivilegedAction<ContainerManager>() { .doAs(new PrivilegedAction<ContainerManager>() {

View File

@ -178,6 +178,10 @@ Release 2.0.5-beta - UNRELEASED
asks an RM to shutdown/resync etc so that NMs can log this message locally asks an RM to shutdown/resync etc so that NMs can log this message locally
for better debuggability. (Mayank Bansal via vinodkv) for better debuggability. (Mayank Bansal via vinodkv)
YARN-617. Made ContainerTokens to be used for validation at NodeManager
also in unsecure mode to prevent AMs from faking resource requirements in
unsecure mode. (Omkar Vinit Joshi via vinodkv)
OPTIMIZATIONS OPTIMIZATIONS
BUG FIXES BUG FIXES

View File

@ -135,13 +135,8 @@ public void init(Configuration conf) {
conf.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true); conf.setBoolean(Dispatcher.DISPATCHER_EXIT_ON_ERROR_KEY, true);
// Create the secretManager if need be. NMContainerTokenSecretManager containerTokenSecretManager =
NMContainerTokenSecretManager containerTokenSecretManager = null; new NMContainerTokenSecretManager(conf);
if (UserGroupInformation.isSecurityEnabled()) {
LOG.info("Security is enabled on NodeManager. "
+ "Creating ContainerTokenSecretManager");
containerTokenSecretManager = new NMContainerTokenSecretManager(conf);
}
this.context = createNMContext(containerTokenSecretManager); this.context = createNMContext(containerTokenSecretManager);

View File

@ -195,15 +195,11 @@ protected void rebootNodeStatusUpdater() {
} }
} }
private boolean isSecurityEnabled() {
return UserGroupInformation.isSecurityEnabled();
}
@Private @Private
protected boolean isTokenKeepAliveEnabled(Configuration conf) { protected boolean isTokenKeepAliveEnabled(Configuration conf) {
return conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED, return conf.getBoolean(YarnConfiguration.LOG_AGGREGATION_ENABLED,
YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED) YarnConfiguration.DEFAULT_LOG_AGGREGATION_ENABLED)
&& isSecurityEnabled(); && UserGroupInformation.isSecurityEnabled();
} }
protected ResourceTracker getRMClient() { protected ResourceTracker getRMClient() {
@ -303,17 +299,14 @@ protected void registerWithRM() throws YarnRemoteException, IOException {
+ message); + message);
} }
if (UserGroupInformation.isSecurityEnabled()) {
MasterKey masterKey = regNMResponse.getMasterKey(); MasterKey masterKey = regNMResponse.getMasterKey();
// do this now so that its set before we start heartbeating to RM // do this now so that its set before we start heartbeating to RM
LOG.info("Security enabled - updating secret keys now");
// It is expected that status updater is started by this point and // It is expected that status updater is started by this point and
// RM gives the shared secret in registration during // RM gives the shared secret in registration during
// StatusUpdater#start(). // StatusUpdater#start().
if (masterKey != null) { if (masterKey != null) {
this.context.getContainerTokenSecretManager().setMasterKey(masterKey); this.context.getContainerTokenSecretManager().setMasterKey(masterKey);
} }
}
LOG.info("Registered with ResourceManager as " + this.nodeId LOG.info("Registered with ResourceManager as " + this.nodeId
+ " with total resource of " + this.totalResource); + " with total resource of " + this.totalResource);
@ -443,10 +436,8 @@ public void run() {
NodeHeartbeatRequest request = recordFactory NodeHeartbeatRequest request = recordFactory
.newRecordInstance(NodeHeartbeatRequest.class); .newRecordInstance(NodeHeartbeatRequest.class);
request.setNodeStatus(nodeStatus); request.setNodeStatus(nodeStatus);
if (isSecurityEnabled()) {
request.setLastKnownMasterKey(NodeStatusUpdaterImpl.this.context request.setLastKnownMasterKey(NodeStatusUpdaterImpl.this.context
.getContainerTokenSecretManager().getCurrentKey()); .getContainerTokenSecretManager().getCurrentKey());
}
while (!isStopped) { while (!isStopped) {
try { try {
rmRetryCount++; rmRetryCount++;
@ -475,14 +466,12 @@ public void run() {
//get next heartbeat interval from response //get next heartbeat interval from response
nextHeartBeatInterval = response.getNextHeartBeatInterval(); nextHeartBeatInterval = response.getNextHeartBeatInterval();
// See if the master-key has rolled over // See if the master-key has rolled over
if (isSecurityEnabled()) {
MasterKey updatedMasterKey = response.getMasterKey(); MasterKey updatedMasterKey = response.getMasterKey();
if (updatedMasterKey != null) { if (updatedMasterKey != null) {
// Will be non-null only on roll-over on RM side // Will be non-null only on roll-over on RM side
context.getContainerTokenSecretManager().setMasterKey( context.getContainerTokenSecretManager().setMasterKey(
updatedMasterKey); updatedMasterKey);
} }
}
if (response.getNodeAction() == NodeAction.SHUTDOWN) { if (response.getNodeAction() == NodeAction.SHUTDOWN) {
LOG LOG

View File

@ -20,6 +20,8 @@
import static org.apache.hadoop.yarn.service.Service.STATE.STARTED; import static org.apache.hadoop.yarn.service.Service.STATE.STARTED;
import java.io.ByteArrayInputStream;
import java.io.DataInputStream;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -29,9 +31,11 @@
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.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.CommonConfigurationKeysPublic; import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
import org.apache.hadoop.io.DataInputByteBuffer; import org.apache.hadoop.io.DataInputByteBuffer;
import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.Server; import org.apache.hadoop.ipc.Server;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.Credentials;
@ -40,7 +44,6 @@
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.util.StringUtils; import org.apache.hadoop.util.StringUtils;
import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.ContainerManager;
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.GetContainerStatusResponse;
@ -52,6 +55,7 @@
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.ContainerToken;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
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;
@ -102,6 +106,9 @@
import org.apache.hadoop.yarn.service.Service; import org.apache.hadoop.yarn.service.Service;
import org.apache.hadoop.yarn.service.ServiceStateChangeListener; import org.apache.hadoop.yarn.service.ServiceStateChangeListener;
import com.google.common.annotations.VisibleForTesting;
import com.google.protobuf.RpcUtil;
public class ContainerManagerImpl extends CompositeService implements public class ContainerManagerImpl extends CompositeService implements
ServiceStateChangeListener, ContainerManager, ServiceStateChangeListener, ContainerManager,
EventHandler<ContainerManagerEvent> { EventHandler<ContainerManagerEvent> {
@ -300,6 +307,33 @@ private ContainerTokenIdentifier selectContainerTokenIdentifier(
return resultId; return resultId;
} }
@Private
@VisibleForTesting
protected ContainerTokenIdentifier getContainerTokenIdentifier(
UserGroupInformation remoteUgi,
org.apache.hadoop.yarn.api.records.Container container)
throws YarnRemoteException {
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 {
ContainerToken containerToken = container.getContainerToken();
Token<ContainerTokenIdentifier> token =
new Token<ContainerTokenIdentifier>(containerToken.getIdentifier()
.array(), containerToken.getPassword().array(), new Text(
containerToken.getKind()), new Text(containerToken.getService()));
try {
return token.decodeIdentifier();
} catch (IOException e) {
throw RPCUtil.getRemoteException(e);
}
}
}
/** /**
* Authorize the request. * Authorize the request.
* *
@ -311,16 +345,14 @@ private ContainerTokenIdentifier selectContainerTokenIdentifier(
* ugi corresponding to the remote end making the api-call * ugi corresponding to the remote end making the api-call
* @throws YarnRemoteException * @throws YarnRemoteException
*/ */
private void authorizeRequest(String containerIDStr, @Private
@VisibleForTesting
protected void authorizeRequest(String containerIDStr,
ContainerLaunchContext launchContext, ContainerLaunchContext launchContext,
org.apache.hadoop.yarn.api.records.Container container, org.apache.hadoop.yarn.api.records.Container container,
UserGroupInformation remoteUgi) UserGroupInformation remoteUgi, ContainerTokenIdentifier tokenId)
throws YarnRemoteException { throws YarnRemoteException {
if (!UserGroupInformation.isSecurityEnabled()) {
return;
}
boolean unauthorized = false; boolean unauthorized = false;
StringBuilder messageBuilder = StringBuilder messageBuilder =
new StringBuilder("Unauthorized request to start container. "); new StringBuilder("Unauthorized request to start container. ");
@ -332,37 +364,29 @@ private void authorizeRequest(String containerIDStr,
} else if (launchContext != null) { } else if (launchContext != null) {
// Verify other things also for startContainer() request. // Verify other things also for startContainer() request.
if (LOG.isDebugEnabled()) {
LOG.debug("Number of TokenIdentifiers in the UGI from RPC: "
+ remoteUgi.getTokenIdentifiers().size());
}
// Get the tokenId from the remote user ugi
ContainerTokenIdentifier tokenId =
selectContainerTokenIdentifier(remoteUgi);
if (tokenId == null) { if (tokenId == null) {
unauthorized = true; unauthorized = true;
messageBuilder messageBuilder
.append("\nContainerTokenIdentifier cannot be null! Null found for " .append("\nNo ContainerToken found for " + containerIDStr);
+ containerIDStr);
} else { } else {
// Is the container coming in with correct user-name? // Is the container coming in with correct user-name?
if (!tokenId.getApplicationSubmitter().equals(launchContext.getUser())) { if (!launchContext.getUser().equals(tokenId.getApplicationSubmitter())) {
unauthorized = true; unauthorized = true;
messageBuilder.append("\n Expected user-name " messageBuilder.append("\n Expected user-name "
+ tokenId.getApplicationSubmitter() + " but found " + tokenId.getApplicationSubmitter() + " but found "
+ launchContext.getUser()); + launchContext.getUser());
} }
// Is the container being relaunched? Or RPC layer let startCall with // Is the container being relaunched? Or RPC layer let startCall with
// tokens generated off old-secret through? // tokens generated off old-secret through?
if (!this.context.getContainerTokenSecretManager() if (!this.context.getContainerTokenSecretManager()
.isValidStartContainerRequest(tokenId)) { .isValidStartContainerRequest(tokenId.getContainerID())) {
unauthorized = true; unauthorized = true;
messageBuilder.append("\n Attempt to relaunch the same " + messageBuilder.append("\n Attempt to relaunch the same "
"container with id " + containerIDStr + "."); + "container with id " + containerIDStr + ".");
} }
// Ensure the token is not expired. // Ensure the token is not expired.
@ -375,7 +399,7 @@ private void authorizeRequest(String containerIDStr,
} }
Resource resource = tokenId.getResource(); Resource resource = tokenId.getResource();
if (!resource.equals(container.getResource())) { if (resource == null || !resource.equals(container.getResource())) {
unauthorized = true; unauthorized = true;
messageBuilder.append("\nExpected resource " + resource messageBuilder.append("\nExpected resource " + resource
+ " but found " + container.getResource()); + " but found " + container.getResource());
@ -411,7 +435,10 @@ public StartContainerResponse startContainer(StartContainerRequest request)
String containerIDStr = containerID.toString(); String containerIDStr = containerID.toString();
UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr); UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
authorizeRequest(containerIDStr, launchContext, lauchContainer, remoteUgi); ContainerTokenIdentifier tokenId =
getContainerTokenIdentifier(remoteUgi, lauchContainer);
authorizeRequest(containerIDStr, launchContext, lauchContainer, remoteUgi,
tokenId);
// Is the container coming from unknown RM // Is the container coming from unknown RM
if (lauchContainer.getRMIdentifer() != nodeStatusUpdater if (lauchContainer.getRMIdentifer() != nodeStatusUpdater
@ -476,13 +503,9 @@ public StartContainerResponse startContainer(StartContainerRequest request)
// TODO: Validate the request // TODO: Validate the request
dispatcher.getEventHandler().handle( dispatcher.getEventHandler().handle(
new ApplicationContainerInitEvent(container)); new ApplicationContainerInitEvent(container));
if (UserGroupInformation.isSecurityEnabled()) {
ContainerTokenIdentifier tokenId =
selectContainerTokenIdentifier(remoteUgi);
this.context.getContainerTokenSecretManager().startContainerSuccessful( this.context.getContainerTokenSecretManager().startContainerSuccessful(
tokenId); tokenId);
}
NMAuditLogger.logSuccess(launchContext.getUser(), NMAuditLogger.logSuccess(launchContext.getUser(),
AuditConstants.START_CONTAINER, "ContainerManageImpl", AuditConstants.START_CONTAINER, "ContainerManageImpl",
applicationID, containerID); applicationID, containerID);
@ -511,12 +534,10 @@ public StopContainerResponse stopContainer(StopContainerRequest request)
// TODO: Only the container's owner can kill containers today. // TODO: Only the container's owner can kill containers today.
UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr); UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
authorizeRequest(containerIDStr, null, null, remoteUgi); Container container = this.context.getContainers().get(containerID);
StopContainerResponse response = StopContainerResponse response =
recordFactory.newRecordInstance(StopContainerResponse.class); recordFactory.newRecordInstance(StopContainerResponse.class);
Container container = this.context.getContainers().get(containerID);
if (container == null) { if (container == null) {
LOG.warn("Trying to stop unknown container " + containerID); LOG.warn("Trying to stop unknown container " + containerID);
NMAuditLogger.logFailure("UnknownUser", NMAuditLogger.logFailure("UnknownUser",
@ -526,6 +547,8 @@ public StopContainerResponse stopContainer(StopContainerRequest request)
containerID); containerID);
return response; // Return immediately. return response; // Return immediately.
} }
authorizeRequest(containerIDStr, null, null, remoteUgi,
getContainerTokenIdentifier(remoteUgi, container.getContainer()));
dispatcher.getEventHandler().handle( dispatcher.getEventHandler().handle(
new ContainerKillEvent(containerID, new ContainerKillEvent(containerID,
@ -554,22 +577,22 @@ public GetContainerStatusResponse getContainerStatus(
// TODO: Only the container's owner can get containers' status today. // TODO: Only the container's owner can get containers' status today.
UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr); UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
authorizeRequest(containerIDStr, null, null, remoteUgi);
LOG.info("Getting container-status for " + containerIDStr); LOG.info("Getting container-status for " + containerIDStr);
Container container = this.context.getContainers().get(containerID); Container container = this.context.getContainers().get(containerID);
if (container != null) { if (container == null) {
ContainerStatus containerStatus = container.cloneAndGetContainerStatus();
LOG.info("Returning " + containerStatus);
GetContainerStatusResponse response = recordFactory
.newRecordInstance(GetContainerStatusResponse.class);
response.setStatus(containerStatus);
return response;
}
throw RPCUtil.getRemoteException("Container " + containerIDStr throw RPCUtil.getRemoteException("Container " + containerIDStr
+ " is not handled by this NodeManager"); + " is not handled by this NodeManager");
} }
authorizeRequest(containerIDStr, null, null, remoteUgi,
getContainerTokenIdentifier(remoteUgi, container.getContainer()));
ContainerStatus containerStatus = container.cloneAndGetContainerStatus();
LOG.info("Returning " + containerStatus);
GetContainerStatusResponse response =
recordFactory.newRecordInstance(GetContainerStatusResponse.class);
response.setStatus(containerStatus);
return response;
}
class ContainerEventDispatcher implements EventHandler<ContainerEvent> { class ContainerEventDispatcher implements EventHandler<ContainerEvent> {
@Override @Override

View File

@ -28,7 +28,6 @@
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.security.Credentials; import org.apache.hadoop.security.Credentials;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType; 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.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
@ -394,9 +393,8 @@ static class AppCompletelyDoneTransition implements
public void transition(ApplicationImpl app, ApplicationEvent event) { public void transition(ApplicationImpl app, ApplicationEvent event) {
// Inform the ContainerTokenSecretManager // Inform the ContainerTokenSecretManager
if (UserGroupInformation.isSecurityEnabled()) {
app.context.getContainerTokenSecretManager().appFinished(app.appId); app.context.getContainerTokenSecretManager().appFinished(app.appId);
}
// Inform the logService // Inform the logService
app.dispatcher.getEventHandler().handle( app.dispatcher.getEventHandler().handle(
new LogHandlerAppFinishedEvent(app.appId)); new LogHandlerAppFinishedEvent(app.appId));

View File

@ -27,7 +27,6 @@
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.security.UserGroupInformation;
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;
@ -35,6 +34,8 @@
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 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.
@ -134,10 +135,6 @@ public synchronized byte[] retrievePassword(
*/ */
public synchronized void startContainerSuccessful( public synchronized void startContainerSuccessful(
ContainerTokenIdentifier tokenId) { ContainerTokenIdentifier tokenId) {
if (!UserGroupInformation.isSecurityEnabled()) {
return;
}
int keyId = tokenId.getMasterKeyId(); int keyId = tokenId.getMasterKeyId();
if (currentMasterKey.getMasterKey().getKeyId() == keyId) { if (currentMasterKey.getMasterKey().getKeyId() == keyId) {
addKeyForContainerId(tokenId.getContainerID(), currentMasterKey); addKeyForContainerId(tokenId.getContainerID(), currentMasterKey);
@ -154,8 +151,7 @@ public synchronized void startContainerSuccessful(
* via retrievePassword. * via retrievePassword.
*/ */
public synchronized boolean isValidStartContainerRequest( public synchronized boolean isValidStartContainerRequest(
ContainerTokenIdentifier tokenId) { ContainerId containerID) {
ContainerId containerID = tokenId.getContainerID();
ApplicationId applicationId = ApplicationId applicationId =
containerID.getApplicationAttemptId().getApplicationId(); containerID.getApplicationAttemptId().getApplicationId();
return !this.oldMasterKeys.containsKey(applicationId) return !this.oldMasterKeys.containsKey(applicationId)

View File

@ -26,7 +26,11 @@
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.Path; import org.apache.hadoop.fs.Path;
import org.apache.hadoop.security.UserGroupInformation;
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.YarnRemoteException;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
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;
@ -173,4 +177,23 @@ public void handle(LogHandlerEvent event) {
public void setBlockNewContainerRequests(boolean blockNewContainerRequests) { public void setBlockNewContainerRequests(boolean blockNewContainerRequests) {
// do nothing // do nothing
} }
@Override
protected void authorizeRequest(String containerIDStr,
ContainerLaunchContext launchContext,
org.apache.hadoop.yarn.api.records.Container container,
UserGroupInformation remoteUgi, ContainerTokenIdentifier tokenId)
throws YarnRemoteException {
// do Nothing
}
@Override
protected ContainerTokenIdentifier getContainerTokenIdentifier(
UserGroupInformation remoteUgi,
org.apache.hadoop.yarn.api.records.Container container)
throws YarnRemoteException {
return new ContainerTokenIdentifier(container.getId(),
container.getNodeHttpAddress(), remoteUgi.getUserName(),
container.getResource(), System.currentTimeMillis(), 123);
}
} }

View File

@ -20,6 +20,8 @@
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer;
import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
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;
@ -28,6 +30,8 @@
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
public class LocalRMInterface implements ResourceTracker { public class LocalRMInterface implements ResourceTracker {
@ -38,6 +42,11 @@ public RegisterNodeManagerResponse registerNodeManager(
RegisterNodeManagerRequest request) throws YarnRemoteException, RegisterNodeManagerRequest request) throws YarnRemoteException,
IOException { IOException {
RegisterNodeManagerResponse response = recordFactory.newRecordInstance(RegisterNodeManagerResponse.class); RegisterNodeManagerResponse response = recordFactory.newRecordInstance(RegisterNodeManagerResponse.class);
MasterKey masterKey = new MasterKeyPBImpl();
masterKey.setKeyId(123);
masterKey.setBytes(ByteBuffer.wrap(new byte[] { new Integer(123)
.byteValue() }));
response.setMasterKey(masterKey);
return response; return response;
} }

View File

@ -20,6 +20,8 @@
import java.io.IOException; import java.io.IOException;
import java.nio.ByteBuffer;
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.yarn.event.Dispatcher; import org.apache.hadoop.yarn.event.Dispatcher;
@ -31,7 +33,9 @@
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics; import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils; import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
@ -67,6 +71,11 @@ public RegisterNodeManagerResponse registerNodeManager(
IOException { IOException {
RegisterNodeManagerResponse response = recordFactory RegisterNodeManagerResponse response = recordFactory
.newRecordInstance(RegisterNodeManagerResponse.class); .newRecordInstance(RegisterNodeManagerResponse.class);
MasterKey masterKey = new MasterKeyPBImpl();
masterKey.setKeyId(123);
masterKey.setBytes(ByteBuffer.wrap(new byte[] { new Integer(123)
.byteValue() }));
response.setMasterKey(masterKey);
return response; return response;
} }

View File

@ -18,6 +18,9 @@
package org.apache.hadoop.yarn.server.nodemanager; package org.apache.hadoop.yarn.server.nodemanager;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
@ -45,7 +48,6 @@
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.junit.Test; import org.junit.Test;
import static org.mockito.Mockito.*;
public class TestEventFlow { public class TestEventFlow {
@ -75,6 +77,7 @@ public void testSuccessfulContainerLaunch() throws InterruptedException,
remoteLogDir.mkdir(); remoteLogDir.mkdir();
YarnConfiguration conf = new YarnConfiguration(); YarnConfiguration conf = new YarnConfiguration();
Context context = new NMContext(new NMContainerTokenSecretManager(conf)); Context context = new NMContext(new NMContainerTokenSecretManager(conf));
conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath()); conf.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath());
@ -112,6 +115,9 @@ public long getRMIdentifier() {
DummyContainerManager containerManager = DummyContainerManager containerManager =
new DummyContainerManager(context, exec, del, nodeStatusUpdater, new DummyContainerManager(context, exec, del, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler); metrics, new ApplicationACLsManager(conf), dirsHandler);
nodeStatusUpdater.init(conf);
((NMContext)context).setContainerManager(containerManager);
nodeStatusUpdater.start();
containerManager.init(conf); containerManager.init(conf);
containerManager.start(); containerManager.start();
@ -132,7 +138,6 @@ public long getRMIdentifier() {
when(mockContainer.getResource()).thenReturn(recordFactory when(mockContainer.getResource()).thenReturn(recordFactory
.newRecordInstance(Resource.class)); .newRecordInstance(Resource.class));
when(mockContainer.getRMIdentifer()).thenReturn(SIMULATED_RM_IDENTIFIER); when(mockContainer.getRMIdentifer()).thenReturn(SIMULATED_RM_IDENTIFIER);
launchContext.setUser("testing"); launchContext.setUser("testing");
StartContainerRequest request = StartContainerRequest request =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);

View File

@ -18,10 +18,16 @@
package org.apache.hadoop.yarn.server.nodemanager; package org.apache.hadoop.yarn.server.nodemanager;
import static org.mockito.Mockito.*; import static org.mockito.Matchers.argThat;
import static org.mockito.Matchers.eq;
import static org.mockito.Matchers.isNull;
import static org.mockito.Mockito.spy;
import static org.mockito.Mockito.times;
import static org.mockito.Mockito.verify;
import java.io.File; import java.io.File;
import java.io.IOException; import java.io.IOException;
import java.security.PrivilegedExceptionAction;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
@ -32,6 +38,8 @@
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.yarn.api.ContainerManager;
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;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@ -41,16 +49,17 @@
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.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.YarnRemoteException; import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.ContainerManagerImpl;
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.util.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;
@ -69,7 +78,6 @@ public class TestNodeManagerReboot {
static final String user = System.getProperty("user.name"); static final String user = System.getProperty("user.name");
private FileContext localFS; private FileContext localFS;
private MyNodeManager nm; private MyNodeManager nm;
private DeletionService delService; private DeletionService delService;
static final Log LOG = LogFactory.getLog(TestNodeManagerReboot.class); static final Log LOG = LogFactory.getLog(TestNodeManagerReboot.class);
@ -87,23 +95,25 @@ public void tearDown() throws IOException, InterruptedException {
} }
} }
@Test(timeout = 20000) @Test(timeout = 2000000)
public void testClearLocalDirWhenNodeReboot() throws IOException, public void testClearLocalDirWhenNodeReboot() throws IOException,
YarnRemoteException { YarnRemoteException, InterruptedException {
nm = new MyNodeManager(); nm = new MyNodeManager();
nm.start(); nm.start();
final ContainerManager containerManager = nm.getContainerManager();
// create files under fileCache // create files under fileCache
createFiles(nmLocalDir.getAbsolutePath(), ContainerLocalizer.FILECACHE, 100); createFiles(nmLocalDir.getAbsolutePath(), ContainerLocalizer.FILECACHE, 100);
localResourceDir.mkdirs(); localResourceDir.mkdirs();
ContainerManagerImpl containerManager = nm.getContainerManager();
ContainerLaunchContext containerLaunchContext = ContainerLaunchContext containerLaunchContext =
Records.newRecord(ContainerLaunchContext.class); Records.newRecord(ContainerLaunchContext.class);
// Construct the Container-id // Construct the Container-id
ContainerId cId = createContainerId(); ContainerId cId = createContainerId();
org.apache.hadoop.yarn.api.records.Container mockContainer = org.apache.hadoop.yarn.api.records.Container mockContainer =
mock(org.apache.hadoop.yarn.api.records.Container.class); Records.newRecord(org.apache.hadoop.yarn.api.records.Container.class);
when(mockContainer.getId()).thenReturn(cId); mockContainer.setId(cId);
containerLaunchContext.setUser(user); containerLaunchContext.setUser(user);
@ -123,17 +133,31 @@ public void testClearLocalDirWhenNodeReboot() throws IOException,
new HashMap<String, LocalResource>(); new HashMap<String, LocalResource>();
localResources.put(destinationFile, localResource); localResources.put(destinationFile, localResource);
containerLaunchContext.setLocalResources(localResources); containerLaunchContext.setLocalResources(localResources);
containerLaunchContext.setUser(containerLaunchContext.getUser());
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 resource = Records.newRecord(Resource.class);
resource.setMemory(1024); resource.setMemory(1024);
when(mockContainer.getResource()).thenReturn(resource); mockContainer.setResource(resource);
StartContainerRequest startRequest = NodeId nodeId = BuilderUtils.newNodeId("127.0.0.1", 12345);
mockContainer.setContainerToken(nm.getNMContext()
.getContainerTokenSecretManager()
.createContainerToken(cId, nodeId, user, resource));
mockContainer.setNodeHttpAddress("127.0.0.1");
mockContainer.setNodeId(nodeId);
final StartContainerRequest startRequest =
Records.newRecord(StartContainerRequest.class); Records.newRecord(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainer(mockContainer); startRequest.setContainer(mockContainer);
final UserGroupInformation currentUser = UserGroupInformation
.createRemoteUser(cId.toString());
currentUser.doAs(new PrivilegedExceptionAction<Void>() {
@Override
public Void run() throws YarnRemoteException, IOException {
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
return null;
}
});
GetContainerStatusRequest request = GetContainerStatusRequest request =
Records.newRecord(GetContainerStatusRequest.class); Records.newRecord(GetContainerStatusRequest.class);

View File

@ -18,15 +18,14 @@
package org.apache.hadoop.yarn.server.nodemanager; package org.apache.hadoop.yarn.server.nodemanager;
import static org.mockito.Mockito.mock;
import static org.mockito.Mockito.when;
import java.io.BufferedReader; import java.io.BufferedReader;
import java.io.BufferedWriter; import java.io.BufferedWriter;
import java.io.File; import java.io.File;
import java.io.FileReader; import java.io.FileReader;
import java.io.FileWriter; import java.io.FileWriter;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress;
import java.security.PrivilegedAction;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
@ -34,9 +33,13 @@
import junit.framework.Assert; import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration;
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.net.NetUtils;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.yarn.api.ContainerManager;
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;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@ -46,18 +49,21 @@
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.ContainerState;
import org.apache.hadoop.yarn.api.records.ContainerStatus; import org.apache.hadoop.yarn.api.records.ContainerStatus;
import org.apache.hadoop.yarn.api.records.ContainerToken;
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.NodeId;
import org.apache.hadoop.yarn.api.records.Resource; import org.apache.hadoop.yarn.api.records.Resource;
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.ContainerPBImpl;
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.YarnRemoteException; import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
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.server.nodemanager.containermanager.ContainerManagerImpl; import org.apache.hadoop.yarn.ipc.YarnRPC;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.After; import org.junit.After;
@ -96,7 +102,7 @@ public void tearDown() throws IOException, InterruptedException {
@Test @Test
public void testKillContainersOnShutdown() throws IOException, public void testKillContainersOnShutdown() throws IOException,
YarnRemoteException { YarnRemoteException {
NodeManager nm = getNodeManager(); NodeManager nm = new TestNodeManager();
nm.init(createNMConfig()); nm.init(createNMConfig());
nm.start(); nm.start();
startContainer(nm, localFS, tmpDir, processStartFile); startContainer(nm, localFS, tmpDir, processStartFile);
@ -136,21 +142,20 @@ public void testKillContainersOnShutdown() throws IOException,
public static void startContainer(NodeManager nm, FileContext localFS, public static void startContainer(NodeManager nm, FileContext localFS,
File scriptFileDir, File processStartFile) throws IOException, File scriptFileDir, File processStartFile) throws IOException,
YarnRemoteException { YarnRemoteException {
ContainerManagerImpl containerManager = nm.getContainerManager();
File scriptFile = File scriptFile =
createUnhaltingScriptFile(scriptFileDir, processStartFile); createUnhaltingScriptFile(scriptFileDir, processStartFile);
ContainerLaunchContext containerLaunchContext = ContainerLaunchContext containerLaunchContext =
recordFactory.newRecordInstance(ContainerLaunchContext.class); recordFactory.newRecordInstance(ContainerLaunchContext.class);
Container mockContainer = mock(Container.class); Container mockContainer = new ContainerPBImpl();
// Construct the Container-id // Construct the Container-id
ContainerId cId = createContainerId(); ContainerId cId = createContainerId();
when(mockContainer.getId()).thenReturn(cId); mockContainer.setId(cId);
NodeId nodeId = BuilderUtils.newNodeId("localhost", 1234); NodeId nodeId = BuilderUtils.newNodeId("localhost", 1234);
when(mockContainer.getNodeId()).thenReturn(nodeId); mockContainer.setNodeId(nodeId);
when(mockContainer.getNodeHttpAddress()).thenReturn("localhost:12345"); mockContainer.setNodeHttpAddress("localhost:12345");
containerLaunchContext.setUser(user); containerLaunchContext.setUser(cId.toString());
URL localResourceUri = URL localResourceUri =
ConverterUtils.getYarnUrlFromPath(localFS ConverterUtils.getYarnUrlFromPath(localFS
@ -173,11 +178,28 @@ public static void startContainer(NodeManager nm, FileContext localFS,
commands.add(scriptFile.getAbsolutePath()); commands.add(scriptFile.getAbsolutePath());
containerLaunchContext.setCommands(commands); containerLaunchContext.setCommands(commands);
Resource resource = BuilderUtils.newResource(1024, 1); Resource resource = BuilderUtils.newResource(1024, 1);
when(mockContainer.getResource()).thenReturn(resource); mockContainer.setResource(resource);
mockContainer.setContainerToken(getContainerToken(nm, cId, nodeId,
cId.toString(), resource));
StartContainerRequest startRequest = StartContainerRequest startRequest =
recordFactory.newRecordInstance(StartContainerRequest.class); recordFactory.newRecordInstance(StartContainerRequest.class);
startRequest.setContainerLaunchContext(containerLaunchContext); startRequest.setContainerLaunchContext(containerLaunchContext);
startRequest.setContainer(mockContainer); startRequest.setContainer(mockContainer);
UserGroupInformation currentUser = UserGroupInformation
.createRemoteUser(cId.toString());
ContainerManager containerManager =
currentUser.doAs(new PrivilegedAction<ContainerManager>() {
@Override
public ContainerManager run() {
Configuration conf = new Configuration();
YarnRPC rpc = YarnRPC.create(conf);
InetSocketAddress containerManagerBindAddress =
NetUtils.createSocketAddrForHost("127.0.0.1", 12345);
return (ContainerManager) rpc.getProxy(ContainerManager.class,
containerManagerBindAddress, conf);
}
});
containerManager.startContainer(startRequest); containerManager.startContainer(startRequest);
GetContainerStatusRequest request = GetContainerStatusRequest request =
@ -234,15 +256,24 @@ private static File createUnhaltingScriptFile(File scriptFileDir,
return scriptFile; return scriptFile;
} }
private NodeManager getNodeManager() { public static ContainerToken getContainerToken(NodeManager nm,
return new NodeManager() { ContainerId containerId, NodeId nodeId, String user, Resource resource) {
return nm.getNMContext().getContainerTokenSecretManager()
.createContainerToken(containerId, nodeId, user, resource);
}
class TestNodeManager extends NodeManager {
@Override @Override
protected NodeStatusUpdater createNodeStatusUpdater(Context context, protected NodeStatusUpdater createNodeStatusUpdater(Context context,
Dispatcher dispatcher, NodeHealthCheckerService healthChecker) { Dispatcher dispatcher, NodeHealthCheckerService healthChecker) {
MockNodeStatusUpdater myNodeStatusUpdater = new MockNodeStatusUpdater( MockNodeStatusUpdater myNodeStatusUpdater =
context, dispatcher, healthChecker, metrics); new MockNodeStatusUpdater(context, dispatcher, healthChecker, metrics);
return myNodeStatusUpdater; return myNodeStatusUpdater;
} }
};
public void setMasterKey(MasterKey masterKey) {
getNMContext().getContainerTokenSecretManager().setMasterKey(masterKey);
}
} }
} }

View File

@ -23,6 +23,7 @@
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections; import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
@ -42,7 +43,6 @@
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.yarn.YarnException; import org.apache.hadoop.yarn.YarnException;
import org.apache.hadoop.yarn.api.ContainerManager;
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.ContainerId; import org.apache.hadoop.yarn.api.records.ContainerId;
@ -63,8 +63,10 @@
import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;
import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse; import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerResponse;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction; import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
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.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;
@ -95,7 +97,7 @@ public class TestNodeStatusUpdater {
private static final RecordFactory recordFactory = RecordFactoryProvider private static final RecordFactory recordFactory = RecordFactoryProvider
.getRecordFactory(null); .getRecordFactory(null);
int heartBeatID = 0; volatile int heartBeatID = 0;
volatile Throwable nmStartError = null; volatile Throwable nmStartError = null;
private final List<NodeId> registeredNodes = new ArrayList<NodeId>(); private final List<NodeId> registeredNodes = new ArrayList<NodeId>();
private final Configuration conf = createNMConfig(); private final Configuration conf = createNMConfig();
@ -113,6 +115,14 @@ public void tearDown() {
DefaultMetricsSystem.shutdown(); DefaultMetricsSystem.shutdown();
} }
public static MasterKey createMasterKey() {
MasterKey masterKey = new MasterKeyPBImpl();
masterKey.setKeyId(123);
masterKey.setBytes(ByteBuffer.wrap(new byte[] { new Integer(123)
.byteValue() }));
return masterKey;
}
private class MyResourceTracker implements ResourceTracker { private class MyResourceTracker implements ResourceTracker {
private final Context context; private final Context context;
@ -137,6 +147,7 @@ public RegisterNodeManagerResponse registerNodeManager(
RegisterNodeManagerResponse response = recordFactory RegisterNodeManagerResponse response = recordFactory
.newRecordInstance(RegisterNodeManagerResponse.class); .newRecordInstance(RegisterNodeManagerResponse.class);
response.setMasterKey(createMasterKey());
return response; return response;
} }
@ -398,6 +409,7 @@ public RegisterNodeManagerResponse registerNodeManager(
RegisterNodeManagerResponse response = recordFactory RegisterNodeManagerResponse response = recordFactory
.newRecordInstance(RegisterNodeManagerResponse.class); .newRecordInstance(RegisterNodeManagerResponse.class);
response.setNodeAction(registerNodeAction ); response.setNodeAction(registerNodeAction );
response.setMasterKey(createMasterKey());
response.setDiagnosticsMessage(shutDownMessage); response.setDiagnosticsMessage(shutDownMessage);
return response; return response;
} }
@ -435,6 +447,7 @@ public RegisterNodeManagerResponse registerNodeManager(
RegisterNodeManagerResponse response = RegisterNodeManagerResponse response =
recordFactory.newRecordInstance(RegisterNodeManagerResponse.class); recordFactory.newRecordInstance(RegisterNodeManagerResponse.class);
response.setNodeAction(registerNodeAction); response.setNodeAction(registerNodeAction);
response.setMasterKey(createMasterKey());
return response; return response;
} }
@ -485,6 +498,7 @@ public RegisterNodeManagerResponse registerNodeManager(
RegisterNodeManagerResponse response = recordFactory RegisterNodeManagerResponse response = recordFactory
.newRecordInstance(RegisterNodeManagerResponse.class); .newRecordInstance(RegisterNodeManagerResponse.class);
response.setNodeAction(registerNodeAction); response.setNodeAction(registerNodeAction);
response.setMasterKey(createMasterKey());
return response; return response;
} }
@ -577,6 +591,8 @@ public RegisterNodeManagerResponse registerNodeManager(
RegisterNodeManagerResponse response = recordFactory RegisterNodeManagerResponse response = recordFactory
.newRecordInstance(RegisterNodeManagerResponse.class); .newRecordInstance(RegisterNodeManagerResponse.class);
response.setNodeAction(registerNodeAction ); response.setNodeAction(registerNodeAction );
response.setMasterKey(createMasterKey());
return response; return response;
} }
@ -635,13 +651,13 @@ public void run() {
+ nm.getServiceState()); + nm.getServiceState());
int waitCount = 0; int waitCount = 0;
while (nm.getServiceState() == STATE.INITED && waitCount++ != 20) { while (nm.getServiceState() == STATE.INITED && waitCount++ != 50) {
LOG.info("Waiting for NM to start.."); LOG.info("Waiting for NM to start..");
if (nmStartError != null) { if (nmStartError != null) {
LOG.error("Error during startup. ", nmStartError); LOG.error("Error during startup. ", nmStartError);
Assert.fail(nmStartError.getCause().getMessage()); Assert.fail(nmStartError.getCause().getMessage());
} }
Thread.sleep(1000); Thread.sleep(2000);
} }
if (nm.getServiceState() != STATE.STARTED) { if (nm.getServiceState() != STATE.STARTED) {
// NM could have failed. // NM could have failed.
@ -686,7 +702,7 @@ protected void cleanupContainers(NodeManagerEventType eventType) {
nm.start(); nm.start();
int waitCount = 0; int waitCount = 0;
while (heartBeatID < 1 && waitCount++ != 20) { while (heartBeatID < 1 && waitCount++ != 200) {
Thread.sleep(500); Thread.sleep(500);
} }
Assert.assertFalse(heartBeatID < 1); Assert.assertFalse(heartBeatID < 1);
@ -714,7 +730,7 @@ public void testNodeDecommision() throws Exception {
nm.start(); nm.start();
int waitCount = 0; int waitCount = 0;
while (heartBeatID < 1 && waitCount++ != 20) { while (heartBeatID < 1 && waitCount++ != 200) {
Thread.sleep(500); Thread.sleep(500);
} }
Assert.assertFalse(heartBeatID < 1); Assert.assertFalse(heartBeatID < 1);
@ -751,9 +767,9 @@ protected NodeStatusUpdater createNodeStatusUpdater(Context context,
+ "Message from ResourceManager: RM Shutting Down Node"); + "Message from ResourceManager: RM Shutting Down Node");
} }
@Test (timeout = 15000) @Test (timeout = 150000)
public void testNMConnectionToRM() { public void testNMConnectionToRM() {
final long delta = 1500; final long delta = 50000;
final long connectionWaitSecs = 5; final long connectionWaitSecs = 5;
final long connectionRetryIntervalSecs = 1; final long connectionRetryIntervalSecs = 1;
//Waiting for rmStartIntervalMS, RM will be started //Waiting for rmStartIntervalMS, RM will be started
@ -891,7 +907,7 @@ public void testApplicationKeepAlive() throws Exception {
/** /**
* Test completed containerStatus get back up when heart beat lost * Test completed containerStatus get back up when heart beat lost
*/ */
@Test(timeout = 20000) @Test(timeout = 200000)
public void testCompletedContainerStatusBackup() throws Exception { public void testCompletedContainerStatusBackup() throws Exception {
nm = new NodeManager() { nm = new NodeManager() {
@Override @Override
@ -925,7 +941,7 @@ protected NMContext createNMContext(
nm.stop(); nm.stop();
} }
@Test(timeout = 20000) @Test(timeout = 200000)
public void testNodeStatusUpdaterRetryAndNMShutdown() public void testNodeStatusUpdaterRetryAndNMShutdown()
throws InterruptedException { throws InterruptedException {
final long connectionWaitSecs = 1; final long connectionWaitSecs = 1;

View File

@ -29,10 +29,13 @@
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.yarn.api.ContainerManager; import org.apache.hadoop.yarn.api.ContainerManager;
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.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.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.conf.YarnConfiguration; import org.apache.hadoop.yarn.conf.YarnConfiguration;
@ -40,6 +43,7 @@
import org.apache.hadoop.yarn.exceptions.YarnRemoteException; import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
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.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;
@ -147,7 +151,52 @@ public void setup() throws IOException {
conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath()); conf.set(YarnConfiguration.NM_REMOTE_APP_LOG_DIR, remoteLogDir.getAbsolutePath());
// Default delSrvc // Default delSrvc
delSrvc = new DeletionService(exec) { delSrvc = createDeletionService();
delSrvc.init(conf);
exec = createContainerExecutor();
nodeHealthChecker = new NodeHealthCheckerService();
nodeHealthChecker.init(conf);
dirsHandler = nodeHealthChecker.getDiskHandler();
containerManager = createContainerManager(delSrvc);
((NMContext)context).setContainerManager(containerManager);
nodeStatusUpdater.init(conf);
containerManager.init(conf);
nodeStatusUpdater.start();
}
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 void authorizeRequest(String containerIDStr,
ContainerLaunchContext launchContext, Container container,
UserGroupInformation remoteUgi, ContainerTokenIdentifier tokenId)
throws YarnRemoteException {
// do nothing
}
@Override
protected ContainerTokenIdentifier getContainerTokenIdentifier(
UserGroupInformation remoteUgi,
org.apache.hadoop.yarn.api.records.Container container)
throws YarnRemoteException {
return new ContainerTokenIdentifier(container.getId(),
container.getNodeHttpAddress(), remoteUgi.getUserName(),
container.getResource(), System.currentTimeMillis(), 123);
}
};
}
protected DeletionService createDeletionService() {
return new DeletionService(exec) {
@Override @Override
public void delete(String user, Path subDir, Path[] baseDirs) { public void delete(String user, Path subDir, Path[] baseDirs) {
// Don't do any deletions. // Don't do any deletions.
@ -155,22 +204,6 @@ public void delete(String user, Path subDir, Path[] baseDirs) {
+ ", baseDirs - " + baseDirs); + ", baseDirs - " + baseDirs);
}; };
}; };
delSrvc.init(conf);
exec = createContainerExecutor();
nodeHealthChecker = new NodeHealthCheckerService();
nodeHealthChecker.init(conf);
dirsHandler = nodeHealthChecker.getDiskHandler();
containerManager =
new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler) {
@Override
public void setBlockNewContainerRequests(
boolean blockNewContainerRequests) {
// do nothing
}
};
containerManager.init(conf);
} }
@After @After

View File

@ -62,7 +62,6 @@
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.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.ConverterUtils; import org.apache.hadoop.yarn.util.ConverterUtils;
import org.junit.Test; import org.junit.Test;
@ -411,15 +410,7 @@ public void testLocalFilesCleanup() throws InterruptedException,
delSrvc = new DeletionService(exec); delSrvc = new DeletionService(exec);
delSrvc.init(conf); delSrvc.init(conf);
containerManager = containerManager = createContainerManager(delSrvc);
new ContainerManagerImpl(context, exec, delSrvc, nodeStatusUpdater,
metrics, new ApplicationACLsManager(conf), dirsHandler) {
@Override
public void setBlockNewContainerRequests(
boolean blockNewContainerRequests) {
// do nothing
}
};
containerManager.init(conf); containerManager.init(conf);
containerManager.start(); containerManager.start();

View File

@ -25,10 +25,13 @@
import static org.mockito.Mockito.verify; import static org.mockito.Mockito.verify;
import static org.mockito.Mockito.when; import static org.mockito.Mockito.when;
import java.nio.ByteBuffer;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import junit.framework.Assert;
import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.yarn.api.records.ApplicationAccessType; import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
import org.apache.hadoop.yarn.api.records.ApplicationAttemptId; import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
@ -37,6 +40,9 @@
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext; import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
import org.apache.hadoop.yarn.event.DrainDispatcher; import org.apache.hadoop.yarn.event.DrainDispatcher;
import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.event.EventHandler;
import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.impl.pb.MasterKeyPBImpl;
import org.apache.hadoop.yarn.server.nodemanager.Context; import org.apache.hadoop.yarn.server.nodemanager.Context;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.AuxServicesEventType;
@ -54,11 +60,13 @@
import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.loghandler.event.LogHandlerEventType;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEvent; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEvent;
import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType; import org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor.ContainersMonitorEventType;
import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
import org.apache.hadoop.yarn.server.security.ApplicationACLsManager; import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.BuilderUtils;
import org.junit.Test; import org.junit.Test;
import org.mockito.ArgumentMatcher; import org.mockito.ArgumentMatcher;
public class TestApplication { public class TestApplication {
/** /**
@ -257,6 +265,10 @@ public void testAppFinishedOnRunningContainers() {
AuxServicesEventType.APPLICATION_STOP, wa.appId))); AuxServicesEventType.APPLICATION_STOP, wa.appId)));
wa.appResourcesCleanedup(); wa.appResourcesCleanedup();
for ( Container container : wa.containers) {
Assert.assertTrue(wa.context.getContainerTokenSecretManager()
.isValidStartContainerRequest(container.getContainer().getId()));
}
assertEquals(ApplicationState.FINISHED, wa.app.getApplicationState()); assertEquals(ApplicationState.FINISHED, wa.app.getApplicationState());
} finally { } finally {
@ -293,6 +305,10 @@ public void testAppFinishedOnCompletedContainers() {
LocalizationEventType.DESTROY_APPLICATION_RESOURCES, wa.app))); LocalizationEventType.DESTROY_APPLICATION_RESOURCES, wa.app)));
wa.appResourcesCleanedup(); wa.appResourcesCleanedup();
for ( Container container : wa.containers) {
Assert.assertTrue(wa.context.getContainerTokenSecretManager()
.isValidStartContainerRequest(container.getContainer().getId()));
}
assertEquals(ApplicationState.FINISHED, wa.app.getApplicationState()); assertEquals(ApplicationState.FINISHED, wa.app.getApplicationState());
} finally { } finally {
if (wa != null) if (wa != null)
@ -429,8 +445,10 @@ private class WrappedApplication {
final Application app; final Application app;
WrappedApplication(int id, long timestamp, String user, int numContainers) { WrappedApplication(int id, long timestamp, String user, int numContainers) {
Configuration conf = new Configuration();
dispatcher = new DrainDispatcher(); dispatcher = new DrainDispatcher();
dispatcher.init(new Configuration()); dispatcher.init(conf);
localizerBus = mock(EventHandler.class); localizerBus = mock(EventHandler.class);
launcherBus = mock(EventHandler.class); launcherBus = mock(EventHandler.class);
@ -448,6 +466,16 @@ private class WrappedApplication {
context = mock(Context.class); context = mock(Context.class);
when(context.getContainerTokenSecretManager()).thenReturn(
new NMContainerTokenSecretManager(conf));
// Setting master key
MasterKey masterKey = new MasterKeyPBImpl();
masterKey.setKeyId(123);
masterKey.setBytes(ByteBuffer.wrap(new byte[] { (new Integer(123)
.byteValue()) }));
context.getContainerTokenSecretManager().setMasterKey(masterKey);
this.user = user; this.user = user;
this.appId = BuilderUtils.newApplicationId(timestamp, id); this.appId = BuilderUtils.newApplicationId(timestamp, id);
@ -455,7 +483,13 @@ private class WrappedApplication {
new Configuration()), this.user, appId, null, context); new Configuration()), this.user, appId, null, context);
containers = new ArrayList<Container>(); containers = new ArrayList<Container>();
for (int i = 0; i < numContainers; i++) { for (int i = 0; i < numContainers; i++) {
containers.add(createMockedContainer(this.appId, i)); Container container = createMockedContainer(this.appId, i);
containers.add(container);
context.getContainerTokenSecretManager().startContainerSuccessful(
new ContainerTokenIdentifier(container.getContainer().getId(), "",
"", null, System.currentTimeMillis() + 1000, masterKey.getKeyId()));
Assert.assertFalse(context.getContainerTokenSecretManager()
.isValidStartContainerRequest(container.getContainer().getId()));
} }
dispatcher.start(); dispatcher.start();

View File

@ -0,0 +1,28 @@
#
# 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.
#
[libdefaults]
default_realm = APACHE.ORG
udp_preference_limit = 1
extra_addresses = 127.0.0.1
[realms]
APACHE.ORG = {
admin_server = localhost:88
kdc = localhost:88
}
[domain_realm]
localhost = APACHE.ORG

View File

@ -44,7 +44,6 @@
import org.apache.hadoop.yarn.server.api.records.MasterKey; import org.apache.hadoop.yarn.server.api.records.MasterKey;
import org.apache.hadoop.yarn.server.api.records.NodeAction; import org.apache.hadoop.yarn.server.api.records.NodeAction;
import org.apache.hadoop.yarn.server.api.records.NodeStatus; import org.apache.hadoop.yarn.server.api.records.NodeStatus;
import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode; 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;
@ -198,11 +197,9 @@ public RegisterNodeManagerResponse registerNodeManager(
return response; return response;
} }
if (isSecurityEnabled()) {
MasterKey nextMasterKeyForNode = MasterKey nextMasterKeyForNode =
this.containerTokenSecretManager.getCurrentKey(); this.containerTokenSecretManager.getCurrentKey();
response.setMasterKey(nextMasterKeyForNode); response.setMasterKey(nextMasterKeyForNode);
}
RMNode rmNode = new RMNodeImpl(nodeId, rmContext, host, cmPort, httpPort, RMNode rmNode = new RMNodeImpl(nodeId, rmContext, host, cmPort, httpPort,
resolve(host), capability); resolve(host), capability);
@ -298,10 +295,9 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
getResponseId() + 1, NodeAction.NORMAL, null, null, null, getResponseId() + 1, NodeAction.NORMAL, null, null, null,
nextHeartBeatInterval); nextHeartBeatInterval);
rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse); rmNode.updateNodeHeartbeatResponseForCleanup(nodeHeartBeatResponse);
// Check if node's masterKey needs to be updated and if the currentKey has // Check if node's masterKey needs to be updated and if the currentKey has
// roller over, send it across // roller over, send it across
if (isSecurityEnabled()) {
boolean shouldSendMasterKey = false; boolean shouldSendMasterKey = false;
MasterKey nextMasterKeyForNode = MasterKey nextMasterKeyForNode =
@ -317,7 +313,6 @@ public NodeHeartbeatResponse nodeHeartbeat(NodeHeartbeatRequest request)
if (shouldSendMasterKey) { if (shouldSendMasterKey) {
nodeHeartBeatResponse.setMasterKey(nextMasterKeyForNode); nodeHeartBeatResponse.setMasterKey(nextMasterKeyForNode);
} }
}
// 4. Send status to RMNode, saving the latest response. // 4. Send status to RMNode, saving the latest response.
this.rmContext.getDispatcher().getEventHandler().handle( this.rmContext.getDispatcher().getEventHandler().handle(
@ -341,8 +336,4 @@ void refreshServiceAcls(Configuration configuration,
PolicyProvider policyProvider) { PolicyProvider policyProvider) {
this.server.refreshServiceAcl(configuration, policyProvider); this.server.refreshServiceAcl(configuration, policyProvider);
} }
protected boolean isSecurityEnabled() {
return UserGroupInformation.isSecurityEnabled();
}
} }

View File

@ -1296,8 +1296,6 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod
unreserve(application, priority, node, rmContainer); unreserve(application, priority, node, rmContainer);
} }
// Create container tokens in secure-mode
if (UserGroupInformation.isSecurityEnabled()) {
ContainerToken containerToken = ContainerToken containerToken =
createContainerToken(application, container); createContainerToken(application, container);
if (containerToken == null) { if (containerToken == null) {
@ -1305,7 +1303,6 @@ private Resource assignContainer(Resource clusterResource, FiCaSchedulerNode nod
return Resources.none(); return Resources.none();
} }
container.setContainerToken(containerToken); container.setContainerToken(containerToken);
}
// Inform the application // Inform the application
RMContainer allocatedContainer = RMContainer allocatedContainer =

View File

@ -25,7 +25,6 @@
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.classification.InterfaceStability.Unstable; import org.apache.hadoop.classification.InterfaceStability.Unstable;
import org.apache.hadoop.security.UserGroupInformation;
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.ContainerToken; import org.apache.hadoop.yarn.api.records.ContainerToken;
@ -35,8 +34,8 @@
import org.apache.hadoop.yarn.api.records.ResourceRequest; import org.apache.hadoop.yarn.api.records.ResourceRequest;
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.server.resourcemanager.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager; import org.apache.hadoop.yarn.server.resourcemanager.ResourceManager;
import org.apache.hadoop.yarn.server.resourcemanager.resource.DefaultResourceCalculator;
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; 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.RMContainer;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.NodeType;
@ -159,17 +158,12 @@ public Container createContainer(
NodeId nodeId = node.getRMNode().getNodeID(); NodeId nodeId = node.getRMNode().getNodeID();
ContainerId containerId = BuilderUtils.newContainerId(application ContainerId containerId = BuilderUtils.newContainerId(application
.getApplicationAttemptId(), application.getNewContainerId()); .getApplicationAttemptId(), application.getNewContainerId());
ContainerToken containerToken = null; ContainerToken containerToken =
// If security is enabled, send the container-tokens too.
if (UserGroupInformation.isSecurityEnabled()) {
containerToken =
containerTokenSecretManager.createContainerToken(containerId, nodeId, containerTokenSecretManager.createContainerToken(containerId, nodeId,
application.getUser(), capability); application.getUser(), capability);
if (containerToken == null) { if (containerToken == null) {
return null; // Try again later. return null; // Try again later.
} }
}
// Create the container // Create the container
Container container = BuilderUtils.newContainer(containerId, nodeId, Container container = BuilderUtils.newContainer(containerId, nodeId,

View File

@ -552,8 +552,6 @@ private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application
.getApplicationAttemptId(), application.getNewContainerId()); .getApplicationAttemptId(), application.getNewContainerId());
ContainerToken containerToken = null; ContainerToken containerToken = null;
// If security is enabled, send the container-tokens too.
if (UserGroupInformation.isSecurityEnabled()) {
containerToken = containerToken =
this.rmContext.getContainerTokenSecretManager() this.rmContext.getContainerTokenSecretManager()
.createContainerToken(containerId, nodeId, .createContainerToken(containerId, nodeId,
@ -561,7 +559,6 @@ private int assignContainer(FiCaSchedulerNode node, FiCaSchedulerApp application
if (containerToken == null) { if (containerToken == null) {
return i; // Try again later. return i; // Try again later.
} }
}
// Create the container // Create the container
Container container = BuilderUtils.newContainer(containerId, nodeId, Container container = BuilderUtils.newContainer(containerId, nodeId,

View File

@ -202,15 +202,18 @@ public List<UpdatedContainerInfo> pullContainerUpdates() {
}; };
private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) { private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr) {
return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++); return buildRMNode(rack, perNode, state, httpAddr, NODE_ID++, null);
} }
private static RMNode buildRMNode(int rack, final Resource perNode, NodeState state, String httpAddr, int hostnum) { private static RMNode buildRMNode(int rack, final Resource perNode,
NodeState state, String httpAddr, int hostnum, String hostName) {
final String rackName = "rack"+ rack; final String rackName = "rack"+ rack;
final int nid = hostnum; final int nid = hostnum;
final String hostName = "host"+ nid;
final String nodeAddr = hostName + ":" + nid; final String nodeAddr = hostName + ":" + nid;
final int port = 123; final int port = 123;
if (hostName == null) {
hostName = "host"+ nid;
}
final NodeId nodeID = newNodeID(hostName, port); final NodeId nodeID = newNodeID(hostName, port);
final String httpAddress = httpAddr; final String httpAddress = httpAddr;
final NodeHealthStatus nodeHealthStatus = final NodeHealthStatus nodeHealthStatus =
@ -233,6 +236,12 @@ public static RMNode newNodeInfo(int rack, final Resource perNode) {
} }
public static RMNode newNodeInfo(int rack, final Resource perNode, int hostnum) { public static RMNode newNodeInfo(int rack, final Resource perNode, int hostnum) {
return buildRMNode(rack, perNode, null, "localhost:0", hostnum); return buildRMNode(rack, perNode, null, "localhost:0", hostnum, null);
} }
public static RMNode newNodeInfo(int rack, final Resource perNode,
int hostnum, String hostName) {
return buildRMNode(rack, perNode, null, "localhost:0", hostnum, hostName);
}
} }

View File

@ -59,6 +59,7 @@
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.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.apache.log4j.Level; import org.apache.log4j.Level;
@ -298,8 +299,12 @@ public void stop() {
@Override @Override
protected ResourceTrackerService createResourceTrackerService() { protected ResourceTrackerService createResourceTrackerService() {
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(new Configuration());
containerTokenSecretManager.rollMasterKey();
return new ResourceTrackerService(getRMContext(), nodesListManager, return new ResourceTrackerService(getRMContext(), nodesListManager,
this.nmLivelinessMonitor, this.containerTokenSecretManager) { this.nmLivelinessMonitor, containerTokenSecretManager) {
@Override @Override
public void start() { public void start() {
// override to not start rpc handler // override to not start rpc handler

View File

@ -59,7 +59,7 @@ public void testAppCleanup() throws Exception {
MockRM rm = new MockRM(); MockRM rm = new MockRM();
rm.start(); rm.start();
MockNM nm1 = rm.registerNode("h1:1234", 5000); MockNM nm1 = rm.registerNode("127.0.0.1:1234", 5000);
RMApp app = rm.submitApp(2000); RMApp app = rm.submitApp(2000);
@ -72,7 +72,7 @@ public void testAppCleanup() throws Exception {
//request for containers //request for containers
int request = 2; int request = 2;
am.allocate("h1" , 1000, request, am.allocate("127.0.0.1" , 1000, request,
new ArrayList<ContainerId>()); new ArrayList<ContainerId>());
//kick the scheduler //kick the scheduler
@ -147,7 +147,7 @@ protected Dispatcher createDispatcher() {
}; };
rm.start(); rm.start();
MockNM nm1 = rm.registerNode("h1:1234", 5000); MockNM nm1 = rm.registerNode("127.0.0.1:1234", 5000);
RMApp app = rm.submitApp(2000); RMApp app = rm.submitApp(2000);
@ -160,7 +160,7 @@ protected Dispatcher createDispatcher() {
//request for containers //request for containers
int request = 2; int request = 2;
am.allocate("h1" , 1000, request, am.allocate("127.0.0.1" , 1000, request,
new ArrayList<ContainerId>()); new ArrayList<ContainerId>());
dispatcher.await(); dispatcher.await();

View File

@ -112,7 +112,7 @@ public void testAMLaunchAndCleanup() throws Exception {
MockRMWithCustomAMLauncher rm = new MockRMWithCustomAMLauncher( MockRMWithCustomAMLauncher rm = new MockRMWithCustomAMLauncher(
containerManager); containerManager);
rm.start(); rm.start();
MockNM nm1 = rm.registerNode("h1:1234", 5120); MockNM nm1 = rm.registerNode("127.0.0.1:1234", 5120);
RMApp app = rm.submitApp(2000); RMApp app = rm.submitApp(2000);

View File

@ -74,8 +74,8 @@ public void test() throws Exception {
rootLogger.setLevel(Level.DEBUG); rootLogger.setLevel(Level.DEBUG);
MockRM rm = new MockRM(conf); MockRM rm = new MockRM(conf);
rm.start(); rm.start();
MockNM nm1 = rm.registerNode("h1:1234", 6 * GB); MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB);
MockNM nm2 = rm.registerNode("h2:5678", 4 * GB); MockNM nm2 = rm.registerNode("127.0.0.2:5678", 4 * GB);
RMApp app1 = rm.submitApp(2048); RMApp app1 = rm.submitApp(2048);
// kick the scheduling, 2 GB given to AM1, remaining 4GB on nm1 // kick the scheduling, 2 GB given to AM1, remaining 4GB on nm1
@ -98,10 +98,10 @@ public void test() throws Exception {
Assert.assertEquals(2 * GB, report_nm2.getUsedResource().getMemory()); Assert.assertEquals(2 * GB, report_nm2.getUsedResource().getMemory());
// add request for containers // add request for containers
am1.addRequests(new String[] { "h1", "h2" }, GB, 1, 1); am1.addRequests(new String[] { "127.0.0.1", "127.0.0.2" }, GB, 1, 1);
AllocateResponse alloc1Response = am1.schedule(); // send the request AllocateResponse alloc1Response = am1.schedule(); // send the request
// add request for containers // add request for containers
am2.addRequests(new String[] { "h1", "h2" }, 3 * GB, 0, 1); am2.addRequests(new String[] { "127.0.0.1", "127.0.0.2" }, 3 * GB, 0, 1);
AllocateResponse alloc2Response = am2.schedule(); // send the request AllocateResponse alloc2Response = am2.schedule(); // send the request
// kick the scheduler, 1 GB and 3 GB given to AM1 and AM2, remaining 0 // kick the scheduler, 1 GB and 3 GB given to AM1 and AM2, remaining 0
@ -163,7 +163,7 @@ private void testMinimumAllocation(YarnConfiguration conf, int testAlloc)
rm.start(); rm.start();
// Register node1 // Register node1
MockNM nm1 = rm.registerNode("h1:1234", 6 * GB); MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB);
// Submit an application // Submit an application
RMApp app1 = rm.submitApp(testAlloc); RMApp app1 = rm.submitApp(testAlloc);
@ -212,8 +212,10 @@ public void testReconnectedNode() throws Exception {
FifoScheduler fs = new FifoScheduler(); FifoScheduler fs = new FifoScheduler();
fs.reinitialize(conf, null); fs.reinitialize(conf, null);
RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1); RMNode n1 =
RMNode n2 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2); MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1, "127.0.0.2");
RMNode n2 =
MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 2, "127.0.0.3");
fs.handle(new NodeAddedSchedulerEvent(n1)); fs.handle(new NodeAddedSchedulerEvent(n1));
fs.handle(new NodeAddedSchedulerEvent(n2)); fs.handle(new NodeAddedSchedulerEvent(n2));
@ -222,7 +224,8 @@ public void testReconnectedNode() throws Exception {
Assert.assertEquals(6 * GB, fs.getRootQueueMetrics().getAvailableMB()); Assert.assertEquals(6 * GB, fs.getRootQueueMetrics().getAvailableMB());
// reconnect n1 with downgraded memory // reconnect n1 with downgraded memory
n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 1); n1 =
MockNodes.newNodeInfo(0, MockNodes.newResource(2 * GB), 1, "127.0.0.2");
fs.handle(new NodeRemovedSchedulerEvent(n1)); fs.handle(new NodeRemovedSchedulerEvent(n1));
fs.handle(new NodeAddedSchedulerEvent(n1)); fs.handle(new NodeAddedSchedulerEvent(n1));
fs.handle(new NodeUpdateSchedulerEvent(n1)); fs.handle(new NodeUpdateSchedulerEvent(n1));
@ -241,7 +244,8 @@ public void testHeadroom() throws Exception {
FifoScheduler fs = (FifoScheduler) rm.getResourceScheduler(); FifoScheduler fs = (FifoScheduler) rm.getResourceScheduler();
// Add a node // Add a node
RMNode n1 = MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1); RMNode n1 =
MockNodes.newNodeInfo(0, MockNodes.newResource(4 * GB), 1, "127.0.0.2");
fs.handle(new NodeAddedSchedulerEvent(n1)); fs.handle(new NodeAddedSchedulerEvent(n1));
// Add two applications // Add two applications

View File

@ -95,8 +95,10 @@ public void testRMRestart() throws Exception {
// start like normal because state is empty // start like normal because state is empty
rm1.start(); rm1.start();
MockNM nm1 = new MockNM("h1:1234", 15120, rm1.getResourceTrackerService()); MockNM nm1 =
MockNM nm2 = new MockNM("h2:5678", 15120, rm1.getResourceTrackerService()); new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
MockNM nm2 =
new MockNM("127.0.0.2:5678", 15120, rm1.getResourceTrackerService());
nm1.registerNode(); nm1.registerNode();
nm2.registerNode(); // nm2 will not heartbeat with RM1 nm2.registerNode(); // nm2 will not heartbeat with RM1
@ -145,7 +147,7 @@ public void testRMRestart() throws Exception {
am1.registerAppAttempt(); am1.registerAppAttempt();
// AM request for containers // AM request for containers
am1.allocate("h1" , 1000, 1, new ArrayList<ContainerId>()); am1.allocate("127.0.0.1" , 1000, 1, new ArrayList<ContainerId>());
// kick the scheduler // kick the scheduler
nm1.nodeHeartbeat(true); nm1.nodeHeartbeat(true);
List<Container> conts = am1.allocate(new ArrayList<ResourceRequest>(), List<Container> conts = am1.allocate(new ArrayList<ResourceRequest>(),
@ -244,8 +246,8 @@ public void testRMRestart() throws Exception {
Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction()); Assert.assertEquals(NodeAction.RESYNC, hbResponse.getNodeAction());
// new NM to represent NM re-register // new NM to represent NM re-register
nm1 = rm2.registerNode("h1:1234", 15120); nm1 = rm2.registerNode("127.0.0.1:1234", 15120);
nm2 = rm2.registerNode("h2:5678", 15120); nm2 = rm2.registerNode("127.0.0.2:5678", 15120);
// verify no more reboot response sent // verify no more reboot response sent
hbResponse = nm1.nodeHeartbeat(true); hbResponse = nm1.nodeHeartbeat(true);
@ -265,7 +267,8 @@ public void testRMRestart() throws Exception {
// Nodes on which the AM's run // Nodes on which the AM's run
MockNM am1Node = nm1; MockNM am1Node = nm1;
if(attemptState.getMasterContainer().getNodeId().toString().contains("h2")){ if (attemptState.getMasterContainer().getNodeId().toString()
.contains("127.0.0.2")) {
am1Node = nm2; am1Node = nm2;
} }
@ -280,7 +283,8 @@ public void testRMRestart() throws Exception {
attemptState.getMasterContainer().getId()); attemptState.getMasterContainer().getId());
MockNM am2Node = nm1; MockNM am2Node = nm1;
if(attemptState.getMasterContainer().getNodeId().toString().contains("h2")){ if (attemptState.getMasterContainer().getNodeId().toString()
.contains("127.0.0.2")) {
am2Node = nm2; am2Node = nm2;
} }
@ -292,8 +296,8 @@ public void testRMRestart() throws Exception {
am2.registerAppAttempt(); am2.registerAppAttempt();
//request for containers //request for containers
am1.allocate("h1" , 1000, 3, new ArrayList<ContainerId>()); am1.allocate("127.0.0.1" , 1000, 3, new ArrayList<ContainerId>());
am2.allocate("h2" , 1000, 1, new ArrayList<ContainerId>()); am2.allocate("127.0.0.2" , 1000, 1, new ArrayList<ContainerId>());
// verify container allocate continues to work // verify container allocate continues to work
nm1.nodeHeartbeat(true); nm1.nodeHeartbeat(true);
@ -346,7 +350,8 @@ public void testRMRestartOnMaxAppAttempts() throws Exception {
rmState.getApplicationState(); rmState.getApplicationState();
MockRM rm1 = new MockRM(conf, memStore); MockRM rm1 = new MockRM(conf, memStore);
rm1.start(); rm1.start();
MockNM nm1 = new MockNM("h1:1234", 15120, rm1.getResourceTrackerService()); MockNM nm1 =
new MockNM("127.0.0.1:1234", 15120, rm1.getResourceTrackerService());
nm1.registerNode(); nm1.registerNode();
// submit an app with maxAppAttempts equals to 1 // submit an app with maxAppAttempts equals to 1

View File

@ -52,6 +52,7 @@ public void setUp() throws Exception {
Configuration conf = new YarnConfiguration(); Configuration conf = new YarnConfiguration();
resourceManager = new ResourceManager(); resourceManager = new ResourceManager();
resourceManager.init(conf); resourceManager.init(conf);
resourceManager.getRMContainerTokenSecretManager().rollMasterKey();
} }
@After @After

View File

@ -48,13 +48,13 @@ public static void setup() {
ResourceScheduler.class); ResourceScheduler.class);
} }
@Test(timeout = 30000) @Test(timeout = 3000000)
public void testRMIdentifierOnContainerAllocation() throws Exception { public void testRMIdentifierOnContainerAllocation() throws Exception {
MockRM rm = new MockRM(conf); MockRM rm = new MockRM(conf);
rm.start(); rm.start();
// Register node1 // Register node1
MockNM nm1 = rm.registerNode("h1:1234", 6 * GB); MockNM nm1 = rm.registerNode("127.0.0.1:1234", 6 * GB);
// Submit an application // Submit an application
RMApp app1 = rm.submitApp(2048); RMApp app1 = rm.submitApp(2048);
@ -65,7 +65,7 @@ public void testRMIdentifierOnContainerAllocation() throws Exception {
MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId()); MockAM am1 = rm.sendAMLaunched(attempt1.getAppAttemptId());
am1.registerAppAttempt(); am1.registerAppAttempt();
am1.addRequests(new String[] { "h1" }, GB, 1, 1); am1.addRequests(new String[] { "127.0.0.1" }, GB, 1, 1);
AllocateResponse alloc1Response = am1.schedule(); // send the request AllocateResponse alloc1Response = am1.schedule(); // send the request
// kick the scheduler // kick the scheduler

View File

@ -90,10 +90,10 @@ private void syncNodeLost(MockNM nm) throws Exception {
@Test @Test
public void testAMRMUnusableNodes() throws Exception { public void testAMRMUnusableNodes() throws Exception {
MockNM nm1 = rm.registerNode("h1:1234", 10000); MockNM nm1 = rm.registerNode("127.0.0.1:1234", 10000);
MockNM nm2 = rm.registerNode("h2:1234", 10000); MockNM nm2 = rm.registerNode("127.0.0.2:1234", 10000);
MockNM nm3 = rm.registerNode("h3:1234", 10000); MockNM nm3 = rm.registerNode("127.0.0.3:1234", 10000);
MockNM nm4 = rm.registerNode("h4:1234", 10000); MockNM nm4 = rm.registerNode("127.0.0.4:1234", 10000);
RMApp app1 = rm.submitApp(2000); RMApp app1 = rm.submitApp(2000);

View File

@ -59,6 +59,7 @@ public class TestRMNMRPCResponseId {
@Before @Before
public void setUp() { public void setUp() {
Configuration conf = new Configuration();
// Dispatcher that processes events inline // Dispatcher that processes events inline
Dispatcher dispatcher = new InlineDispatcher(); Dispatcher dispatcher = new InlineDispatcher();
dispatcher.register(SchedulerEventType.class, new EventHandler<Event>() { dispatcher.register(SchedulerEventType.class, new EventHandler<Event>() {
@ -69,17 +70,16 @@ public void handle(Event event) {
}); });
RMContext context = RMContext context =
new RMContextImpl(dispatcher, null, null, null, null, new RMContextImpl(dispatcher, null, null, null, null,
null, null, null); null, new RMContainerTokenSecretManager(conf), null);
dispatcher.register(RMNodeEventType.class, dispatcher.register(RMNodeEventType.class,
new ResourceManager.NodeEventDispatcher(context)); new ResourceManager.NodeEventDispatcher(context));
NodesListManager nodesListManager = new NodesListManager(context); NodesListManager nodesListManager = new NodesListManager(context);
Configuration conf = new Configuration();
nodesListManager.init(conf); nodesListManager.init(conf);
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(conf); context.getContainerTokenSecretManager().rollMasterKey();
resourceTrackerService = new ResourceTrackerService(context, resourceTrackerService = new ResourceTrackerService(context,
nodesListManager, new NMLivelinessMonitor(dispatcher), nodesListManager, new NMLivelinessMonitor(dispatcher),
containerTokenSecretManager); context.getContainerTokenSecretManager());
resourceTrackerService.init(conf); resourceTrackerService.init(conf);
} }

View File

@ -52,6 +52,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources; import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -88,6 +89,11 @@ public void setUp() throws IOException {
thenReturn(CapacityScheduler.queueComparator); thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()). when(csContext.getResourceCalculator()).
thenReturn(resourceCalculator); thenReturn(resourceCalculator);
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(conf);
containerTokenSecretManager.rollMasterKey();
when(csContext.getContainerTokenSecretManager()).thenReturn(
containerTokenSecretManager);
Map<String, CSQueue> queues = new HashMap<String, CSQueue>(); Map<String, CSQueue> queues = new HashMap<String, CSQueue>();
CSQueue root = CSQueue root =

View File

@ -67,6 +67,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerApp;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.common.fica.FiCaSchedulerNode;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.AppRemovedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
import org.junit.Test; import org.junit.Test;
@ -122,6 +123,11 @@ public void setUp() throws Exception {
thenReturn(CapacityScheduler.queueComparator); thenReturn(CapacityScheduler.queueComparator);
when(csContext.getResourceCalculator()). when(csContext.getResourceCalculator()).
thenReturn(resourceCalculator); thenReturn(resourceCalculator);
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(conf);
containerTokenSecretManager.rollMasterKey();
when(csContext.getContainerTokenSecretManager()).thenReturn(
containerTokenSecretManager);
root = root =
CapacityScheduler.parseQueue(csContext, csConf, null, CapacityScheduler.parseQueue(csContext, csConf, null,
@ -275,7 +281,7 @@ public void testSingleQueueOneUserMetrics() throws Exception {
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
final int numNodes = 1; final int numNodes = 1;
@ -397,7 +403,7 @@ public void testSingleQueueWithOneUser() throws Exception {
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
final int numNodes = 1; final int numNodes = 1;
@ -528,9 +534,9 @@ public void testUserLimits() throws Exception {
a.submitApplication(app_2, user_1, A); a.submitApplication(app_2, user_1, A);
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
String host_1 = "host_1"; String host_1 = "127.0.0.2";
FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB); FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB);
final int numNodes = 2; final int numNodes = 2;
@ -622,9 +628,9 @@ public void testHeadroomWithMaxCap() throws Exception {
a.submitApplication(app_2, user_1, A); a.submitApplication(app_2, user_1, A);
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
String host_1 = "host_1"; String host_1 = "127.0.0.2";
FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB); FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 8*GB);
final int numNodes = 2; final int numNodes = 2;
@ -740,7 +746,7 @@ public void testSingleQueueWithMultipleUsers() throws Exception {
a.submitApplication(app_3, user_2, A); a.submitApplication(app_3, user_2, A);
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 8*GB);
final int numNodes = 1; final int numNodes = 1;
@ -902,7 +908,7 @@ public void testReservation() throws Exception {
a.submitApplication(app_1, user_1, A); a.submitApplication(app_1, user_1, A);
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB);
final int numNodes = 2; final int numNodes = 2;
@ -1002,9 +1008,9 @@ public void testStolenReservedContainer() throws Exception {
a.submitApplication(app_1, user_1, A); a.submitApplication(app_1, user_1, A);
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB);
String host_1 = "host_1"; String host_1 = "127.0.0.2";
FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB); FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB);
final int numNodes = 3; final int numNodes = 3;
@ -1102,10 +1108,10 @@ public void testReservationExchange() throws Exception {
a.submitApplication(app_1, user_1, A); a.submitApplication(app_1, user_1, A);
// Setup some nodes // Setup some nodes
String host_0 = "host_0"; String host_0 = "127.0.0.1";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, DEFAULT_RACK, 0, 4*GB);
String host_1 = "host_1"; String host_1 = "127.0.0.2";
FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB); FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, DEFAULT_RACK, 0, 4*GB);
final int numNodes = 3; final int numNodes = 3;
@ -1214,15 +1220,15 @@ public void testLocalityScheduling() throws Exception {
a.submitApplication(app_0, user_0, A); a.submitApplication(app_0, user_0, A);
// Setup some nodes and racks // Setup some nodes and racks
String host_0 = "host_0"; String host_0 = "127.0.0.1";
String rack_0 = "rack_0"; String rack_0 = "rack_0";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, rack_0, 0, 8*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, rack_0, 0, 8*GB);
String host_1 = "host_1"; String host_1 = "127.0.0.2";
String rack_1 = "rack_1"; String rack_1 = "rack_1";
FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, rack_1, 0, 8*GB); FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, rack_1, 0, 8*GB);
String host_2 = "host_2"; String host_2 = "127.0.0.3";
String rack_2 = "rack_2"; String rack_2 = "rack_2";
FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB); FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB);
@ -1317,7 +1323,7 @@ public void testLocalityScheduling() throws Exception {
app_0.updateResourceRequests(app_0_requests_0); app_0.updateResourceRequests(app_0_requests_0);
assertEquals(2, app_0.getTotalRequiredResources(priority)); assertEquals(2, app_0.getTotalRequiredResources(priority));
String host_3 = "host_3"; // on rack_1 String host_3 = "127.0.0.4"; // on rack_1
FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB); FiCaSchedulerNode node_3 = TestUtils.getMockNode(host_3, rack_1, 0, 8*GB);
// Rack-delay // Rack-delay
@ -1355,15 +1361,15 @@ public void testApplicationPriorityScheduling() throws Exception {
a.submitApplication(app_0, user_0, A); a.submitApplication(app_0, user_0, A);
// Setup some nodes and racks // Setup some nodes and racks
String host_0 = "host_0"; String host_0 = "127.0.0.1";
String rack_0 = "rack_0"; String rack_0 = "rack_0";
FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, rack_0, 0, 8*GB); FiCaSchedulerNode node_0 = TestUtils.getMockNode(host_0, rack_0, 0, 8*GB);
String host_1 = "host_1"; String host_1 = "127.0.0.2";
String rack_1 = "rack_1"; String rack_1 = "rack_1";
FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, rack_1, 0, 8*GB); FiCaSchedulerNode node_1 = TestUtils.getMockNode(host_1, rack_1, 0, 8*GB);
String host_2 = "host_2"; String host_2 = "127.0.0.3";
String rack_2 = "rack_2"; String rack_2 = "rack_2";
FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB); FiCaSchedulerNode node_2 = TestUtils.getMockNode(host_2, rack_2, 0, 8*GB);
@ -1486,14 +1492,14 @@ public void testSchedulingConstraints() throws Exception {
a.submitApplication(app_0, user_0, A); a.submitApplication(app_0, user_0, A);
// Setup some nodes and racks // Setup some nodes and racks
String host_0_0 = "host_0_0"; String host_0_0 = "127.0.0.1";
String rack_0 = "rack_0"; String rack_0 = "rack_0";
FiCaSchedulerNode node_0_0 = TestUtils.getMockNode(host_0_0, rack_0, 0, 8*GB); FiCaSchedulerNode node_0_0 = TestUtils.getMockNode(host_0_0, rack_0, 0, 8*GB);
String host_0_1 = "host_0_1"; String host_0_1 = "127.0.0.2";
FiCaSchedulerNode node_0_1 = TestUtils.getMockNode(host_0_1, rack_0, 0, 8*GB); FiCaSchedulerNode node_0_1 = TestUtils.getMockNode(host_0_1, rack_0, 0, 8*GB);
String host_1_0 = "host_1_0"; String host_1_0 = "127.0.0.3";
String rack_1 = "rack_1"; String rack_1 = "rack_1";
FiCaSchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB); FiCaSchedulerNode node_1_0 = TestUtils.getMockNode(host_1_0, rack_1, 0, 8*GB);

View File

@ -123,6 +123,8 @@ public void setUp() throws IOException {
resourceManager.init(conf); resourceManager.init(conf);
((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start(); ((AsyncDispatcher)resourceManager.getRMContext().getDispatcher()).start();
scheduler.reinitialize(conf, resourceManager.getRMContext()); scheduler.reinitialize(conf, resourceManager.getRMContext());
// to initialize the master key
resourceManager.getRMContainerTokenSecretManager().rollMasterKey();
} }
@After @After
@ -221,13 +223,16 @@ public void testLoadConfigurationOnInitialize() throws IOException {
@Test @Test
public void testAggregateCapacityTracking() throws Exception { public void testAggregateCapacityTracking() throws Exception {
// Add a node // Add a node
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
assertEquals(1024, scheduler.getClusterCapacity().getMemory()); assertEquals(1024, scheduler.getClusterCapacity().getMemory());
// Add another node // Add another node
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(512)); RMNode node2 =
MockNodes.newNodeInfo(1, Resources.createResource(512), 2, "127.0.0.2");
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
scheduler.handle(nodeEvent2); scheduler.handle(nodeEvent2);
assertEquals(1536, scheduler.getClusterCapacity().getMemory()); assertEquals(1536, scheduler.getClusterCapacity().getMemory());
@ -241,7 +246,9 @@ public void testAggregateCapacityTracking() throws Exception {
@Test @Test
public void testSimpleFairShareCalculation() { public void testSimpleFairShareCalculation() {
// Add one big node (only care about aggregate capacity) // Add one big node (only care about aggregate capacity)
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(10 * 1024)); RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(10 * 1024), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -265,7 +272,9 @@ public void testSimpleFairShareCalculation() {
public void testSimpleHierarchicalFairShareCalculation() { public void testSimpleHierarchicalFairShareCalculation() {
// Add one big node (only care about aggregate capacity) // Add one big node (only care about aggregate capacity)
int capacity = 10 * 24; int capacity = 10 * 24;
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(capacity)); RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(capacity), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -313,12 +322,15 @@ public void testHierarchicalQueuesSimilarParents() {
@Test (timeout = 5000) @Test (timeout = 5000)
public void testSimpleContainerAllocation() { public void testSimpleContainerAllocation() {
// Add a node // Add a node
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
// Add another node // Add another node
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(512)); RMNode node2 =
MockNodes.newNodeInfo(1, Resources.createResource(512), 2, "127.0.0.2");
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
scheduler.handle(nodeEvent2); scheduler.handle(nodeEvent2);
@ -351,7 +363,9 @@ public void testSimpleContainerAllocation() {
@Test (timeout = 5000) @Test (timeout = 5000)
public void testSimpleContainerReservation() throws InterruptedException { public void testSimpleContainerReservation() throws InterruptedException {
// Add a node // Add a node
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -359,6 +373,7 @@ public void testSimpleContainerReservation() throws InterruptedException {
createSchedulingRequest(1024, "queue1", "user1", 1); createSchedulingRequest(1024, "queue1", "user1", 1);
scheduler.update(); scheduler.update();
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node1);
scheduler.handle(updateEvent); scheduler.handle(updateEvent);
// Make sure queue 1 is allocated app capacity // Make sure queue 1 is allocated app capacity
@ -376,7 +391,9 @@ public void testSimpleContainerReservation() throws InterruptedException {
assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory()); assertEquals(1024, scheduler.applications.get(attId).getCurrentReservation().getMemory());
// Now another node checks in with capacity // Now another node checks in with capacity
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node2 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 2, "127.0.0.2");
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2); NodeUpdateSchedulerEvent updateEvent2 = new NodeUpdateSchedulerEvent(node2);
scheduler.handle(nodeEvent2); scheduler.handle(nodeEvent2);
@ -442,7 +459,9 @@ public void testFairShareWithMinAlloc() throws Exception {
queueManager.initialize(); queueManager.initialize();
// Add one big node (only care about aggregate capacity) // Add one big node (only care about aggregate capacity)
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024)); RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(3 * 1024), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -799,7 +818,9 @@ public void testIsStarvedForMinShare() throws Exception {
queueManager.initialize(); queueManager.initialize();
// Add one big node (only care about aggregate capacity) // Add one big node (only care about aggregate capacity)
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024)); RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -857,7 +878,9 @@ public void testIsStarvedForFairShare() throws Exception {
queueManager.initialize(); queueManager.initialize();
// Add one big node (only care about aggregate capacity) // Add one big node (only care about aggregate capacity)
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024)); RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(4 * 1024), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -933,15 +956,21 @@ public void testChoiceOfPreemptedContainers() throws Exception {
queueManager.initialize(); queueManager.initialize();
// Create four nodes // Create four nodes
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); RMNode node2 =
MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024), 2,
"127.0.0.2");
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
scheduler.handle(nodeEvent2); scheduler.handle(nodeEvent2);
RMNode node3 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); RMNode node3 =
MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024), 3,
"127.0.0.3");
NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3); NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3);
scheduler.handle(nodeEvent3); scheduler.handle(nodeEvent3);
@ -1094,15 +1123,21 @@ public void testPreemptionDecision() throws Exception {
queueManager.initialize(); queueManager.initialize();
// Create four nodes // Create four nodes
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); RMNode node1 =
MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024), 1,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); RMNode node2 =
MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024), 2,
"127.0.0.2");
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
scheduler.handle(nodeEvent2); scheduler.handle(nodeEvent2);
RMNode node3 = MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024)); RMNode node3 =
MockNodes.newNodeInfo(1, Resources.createResource(2 * 1024), 3,
"127.0.0.3");
NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3); NodeAddedSchedulerEvent nodeEvent3 = new NodeAddedSchedulerEvent(node3);
scheduler.handle(nodeEvent3); scheduler.handle(nodeEvent3);
@ -1183,7 +1218,9 @@ public void testPreemptionDecision() throws Exception {
@Test (timeout = 5000) @Test (timeout = 5000)
public void testMultipleContainersWaitingForReservation() { public void testMultipleContainersWaitingForReservation() {
// Add a node // Add a node
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -1226,7 +1263,9 @@ public void testUserMaxRunningApps() throws Exception {
queueManager.initialize(); queueManager.initialize();
// Add a node // Add a node
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(8192)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(8192), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -1263,7 +1302,9 @@ public void testUserMaxRunningApps() throws Exception {
@Test (timeout = 5000) @Test (timeout = 5000)
public void testReservationWhileMultiplePriorities() { public void testReservationWhileMultiplePriorities() {
// Add a node // Add a node
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -1348,9 +1389,15 @@ public void testAclSubmitApplication() throws Exception {
@Test (timeout = 5000) @Test (timeout = 5000)
public void testMultipleNodesSingleRackRequest() throws Exception { public void testMultipleNodesSingleRackRequest() throws Exception {
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node1 =
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); MockNodes
RMNode node3 = MockNodes.newNodeInfo(2, Resources.createResource(1024)); .newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
RMNode node2 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 2, "127.0.0.2");
RMNode node3 =
MockNodes
.newNodeInfo(2, Resources.createResource(1024), 3, "127.0.0.3");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
@ -1388,7 +1435,9 @@ public void testMultipleNodesSingleRackRequest() throws Exception {
@Test (timeout = 5000) @Test (timeout = 5000)
public void testFifoWithinQueue() throws Exception { public void testFifoWithinQueue() throws Exception {
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(3072)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(3072), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);
@ -1432,7 +1481,9 @@ public void testMaxAssign() throws AllocationConfigurationException {
scheduler.getQueueManager().getLeafQueue("root.default") scheduler.getQueueManager().getLeafQueue("root.default")
.setPolicy(SchedulingPolicy.getDefault()); .setPolicy(SchedulingPolicy.getDefault());
RMNode node = MockNodes.newNodeInfo(1, Resources.createResource(16384)); RMNode node =
MockNodes.newNodeInfo(1, Resources.createResource(16384), 0,
"127.0.0.1");
NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node); NodeAddedSchedulerEvent nodeEvent = new NodeAddedSchedulerEvent(node);
NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node); NodeUpdateSchedulerEvent updateEvent = new NodeUpdateSchedulerEvent(node);
scheduler.handle(nodeEvent); scheduler.handle(nodeEvent);
@ -1477,8 +1528,12 @@ public void testAssignContainer() throws Exception {
final String fairChild1 = fairParent + ".fairChild1"; final String fairChild1 = fairParent + ".fairChild1";
final String fairChild2 = fairParent + ".fairChild2"; final String fairChild2 = fairParent + ".fairChild2";
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(8192)); RMNode node1 =
RMNode node2 = MockNodes.newNodeInfo(1, Resources.createResource(8192)); MockNodes
.newNodeInfo(1, Resources.createResource(8192), 1, "127.0.0.1");
RMNode node2 =
MockNodes
.newNodeInfo(1, Resources.createResource(8192), 2, "127.0.0.2");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2); NodeAddedSchedulerEvent nodeEvent2 = new NodeAddedSchedulerEvent(node2);
@ -1597,7 +1652,9 @@ public void testNotAllowSubmitApplication() throws Exception {
@Test @Test
public void testReservationThatDoesntFit() { public void testReservationThatDoesntFit() {
RMNode node1 = MockNodes.newNodeInfo(1, Resources.createResource(1024)); RMNode node1 =
MockNodes
.newNodeInfo(1, Resources.createResource(1024), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node1);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);

View File

@ -56,6 +56,7 @@
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeUpdateSchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent; import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.SchedulerEvent;
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.BuilderUtils;
import org.junit.After; import org.junit.After;
import org.junit.Before; import org.junit.Before;
@ -153,14 +154,17 @@ public void testAppAttemptMetrics() throws Exception {
@Test(timeout=2000) @Test(timeout=2000)
public void testNodeLocalAssignment() throws Exception { public void testNodeLocalAssignment() throws Exception {
AsyncDispatcher dispatcher = new InlineDispatcher(); AsyncDispatcher dispatcher = new InlineDispatcher();
RMContainerTokenSecretManager containerTokenSecretManager =
new RMContainerTokenSecretManager(new Configuration());
containerTokenSecretManager.rollMasterKey();
RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null, RMContext rmContext = new RMContextImpl(dispatcher, null, null, null, null,
null, null, null); null, containerTokenSecretManager, null);
FifoScheduler scheduler = new FifoScheduler(); FifoScheduler scheduler = new FifoScheduler();
scheduler.reinitialize(new Configuration(), rmContext); scheduler.reinitialize(new Configuration(), rmContext);
RMNode node0 = MockNodes.newNodeInfo(1, RMNode node0 = MockNodes.newNodeInfo(1,
Resources.createResource(1024 * 64), 1234); Resources.createResource(1024 * 64), 1, "127.0.0.1");
NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node0); NodeAddedSchedulerEvent nodeEvent1 = new NodeAddedSchedulerEvent(node0);
scheduler.handle(nodeEvent1); scheduler.handle(nodeEvent1);

View File

@ -121,7 +121,7 @@ public TestRMWebServicesApps() {
@Test @Test
public void testApps() throws JSONException, Exception { public void testApps() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testAppsHelper("apps", app1, MediaType.APPLICATION_JSON); testAppsHelper("apps", app1, MediaType.APPLICATION_JSON);
@ -131,7 +131,7 @@ public void testApps() throws JSONException, Exception {
@Test @Test
public void testAppsSlash() throws JSONException, Exception { public void testAppsSlash() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testAppsHelper("apps/", app1, MediaType.APPLICATION_JSON); testAppsHelper("apps/", app1, MediaType.APPLICATION_JSON);
@ -141,7 +141,7 @@ public void testAppsSlash() throws JSONException, Exception {
@Test @Test
public void testAppsDefault() throws JSONException, Exception { public void testAppsDefault() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testAppsHelper("apps/", app1, ""); testAppsHelper("apps/", app1, "");
@ -151,7 +151,7 @@ public void testAppsDefault() throws JSONException, Exception {
@Test @Test
public void testAppsXML() throws JSONException, Exception { public void testAppsXML() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024, "testwordcount", "user1"); RMApp app1 = rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -176,7 +176,7 @@ public void testAppsXML() throws JSONException, Exception {
@Test @Test
public void testAppsXMLMulti() throws JSONException, Exception { public void testAppsXMLMulti() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024, "testwordcount", "user1"); rm.submitApp(1024, "testwordcount", "user1");
rm.submitApp(2048, "testwordcount2", "user1"); rm.submitApp(2048, "testwordcount2", "user1");
@ -220,7 +220,7 @@ public void testAppsHelper(String path, RMApp app, String media)
@Test @Test
public void testAppsQueryState() throws JSONException, Exception { public void testAppsQueryState() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -242,7 +242,7 @@ public void testAppsQueryState() throws JSONException, Exception {
@Test @Test
public void testAppsQueryStateNone() throws JSONException, Exception { public void testAppsQueryStateNone() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -260,7 +260,7 @@ public void testAppsQueryStateNone() throws JSONException, Exception {
@Test @Test
public void testAppsQueryStateInvalid() throws JSONException, Exception { public void testAppsQueryStateInvalid() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -298,7 +298,7 @@ public void testAppsQueryStateInvalid() throws JSONException, Exception {
@Test @Test
public void testAppsQueryFinalStatus() throws JSONException, Exception { public void testAppsQueryFinalStatus() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -321,7 +321,7 @@ public void testAppsQueryFinalStatus() throws JSONException, Exception {
@Test @Test
public void testAppsQueryFinalStatusNone() throws JSONException, Exception { public void testAppsQueryFinalStatusNone() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -339,7 +339,7 @@ public void testAppsQueryFinalStatusNone() throws JSONException, Exception {
@Test @Test
public void testAppsQueryFinalStatusInvalid() throws JSONException, Exception { public void testAppsQueryFinalStatusInvalid() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -377,7 +377,7 @@ public void testAppsQueryFinalStatusInvalid() throws JSONException, Exception {
@Test @Test
public void testAppsQueryUser() throws JSONException, Exception { public void testAppsQueryUser() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
rm.submitApp(1024); rm.submitApp(1024);
@ -405,7 +405,7 @@ public void testAppsQueryUser() throws JSONException, Exception {
@Test @Test
public void testAppsQueryQueue() throws JSONException, Exception { public void testAppsQueryQueue() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
rm.submitApp(1024); rm.submitApp(1024);
@ -428,7 +428,7 @@ public void testAppsQueryQueue() throws JSONException, Exception {
@Test @Test
public void testAppsQueryLimit() throws JSONException, Exception { public void testAppsQueryLimit() throws JSONException, Exception {
rm.start(); rm.start();
rm.registerNode("amNM:1234", 2048); rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
rm.submitApp(1024); rm.submitApp(1024);
rm.submitApp(1024); rm.submitApp(1024);
@ -451,7 +451,7 @@ public void testAppsQueryStartBegin() throws JSONException, Exception {
rm.start(); rm.start();
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
Thread.sleep(1); Thread.sleep(1);
rm.registerNode("amNM:1234", 2048); rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
rm.submitApp(1024); rm.submitApp(1024);
rm.submitApp(1024); rm.submitApp(1024);
@ -472,7 +472,7 @@ public void testAppsQueryStartBegin() throws JSONException, Exception {
@Test @Test
public void testAppsQueryStartBeginSome() throws JSONException, Exception { public void testAppsQueryStartBeginSome() throws JSONException, Exception {
rm.start(); rm.start();
rm.registerNode("amNM:1234", 2048); rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
rm.submitApp(1024); rm.submitApp(1024);
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
@ -495,7 +495,7 @@ public void testAppsQueryStartBeginSome() throws JSONException, Exception {
@Test @Test
public void testAppsQueryStartEnd() throws JSONException, Exception { public void testAppsQueryStartEnd() throws JSONException, Exception {
rm.start(); rm.start();
rm.registerNode("amNM:1234", 2048); rm.registerNode("127.0.0.1:1234", 2048);
long end = System.currentTimeMillis(); long end = System.currentTimeMillis();
Thread.sleep(1); Thread.sleep(1);
rm.submitApp(1024); rm.submitApp(1024);
@ -515,7 +515,7 @@ public void testAppsQueryStartEnd() throws JSONException, Exception {
@Test @Test
public void testAppsQueryStartBeginEnd() throws JSONException, Exception { public void testAppsQueryStartBeginEnd() throws JSONException, Exception {
rm.start(); rm.start();
rm.registerNode("amNM:1234", 2048); rm.registerNode("127.0.0.1:1234", 2048);
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
Thread.sleep(1); Thread.sleep(1);
rm.submitApp(1024); rm.submitApp(1024);
@ -541,7 +541,7 @@ public void testAppsQueryStartBeginEnd() throws JSONException, Exception {
@Test @Test
public void testAppsQueryFinishBegin() throws JSONException, Exception { public void testAppsQueryFinishBegin() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
Thread.sleep(1); Thread.sleep(1);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
@ -573,7 +573,7 @@ public void testAppsQueryFinishBegin() throws JSONException, Exception {
@Test @Test
public void testAppsQueryFinishEnd() throws JSONException, Exception { public void testAppsQueryFinishEnd() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
// finish App // finish App
@ -605,7 +605,7 @@ public void testAppsQueryFinishEnd() throws JSONException, Exception {
@Test @Test
public void testAppsQueryFinishBeginEnd() throws JSONException, Exception { public void testAppsQueryFinishBeginEnd() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
long start = System.currentTimeMillis(); long start = System.currentTimeMillis();
Thread.sleep(1); Thread.sleep(1);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
@ -640,7 +640,7 @@ public void testAppsQueryFinishBeginEnd() throws JSONException, Exception {
@Test @Test
public void testSingleApp() throws JSONException, Exception { public void testSingleApp() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024, "testwordcount", "user1"); RMApp app1 = rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testSingleAppsHelper(app1.getApplicationId().toString(), app1, testSingleAppsHelper(app1.getApplicationId().toString(), app1,
@ -651,7 +651,7 @@ public void testSingleApp() throws JSONException, Exception {
@Test @Test
public void testSingleAppsSlash() throws JSONException, Exception { public void testSingleAppsSlash() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testSingleAppsHelper(app1.getApplicationId().toString() + "/", app1, testSingleAppsHelper(app1.getApplicationId().toString() + "/", app1,
@ -662,7 +662,7 @@ public void testSingleAppsSlash() throws JSONException, Exception {
@Test @Test
public void testSingleAppsDefault() throws JSONException, Exception { public void testSingleAppsDefault() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testSingleAppsHelper(app1.getApplicationId().toString() + "/", app1, ""); testSingleAppsHelper(app1.getApplicationId().toString() + "/", app1, "");
@ -672,7 +672,7 @@ public void testSingleAppsDefault() throws JSONException, Exception {
@Test @Test
public void testInvalidApp() throws JSONException, Exception { public void testInvalidApp() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -708,7 +708,7 @@ public void testInvalidApp() throws JSONException, Exception {
@Test @Test
public void testNonexistApp() throws JSONException, Exception { public void testNonexistApp() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024, "testwordcount", "user1"); rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -757,7 +757,7 @@ public void testSingleAppsHelper(String path, RMApp app, String media)
@Test @Test
public void testSingleAppsXML() throws JSONException, Exception { public void testSingleAppsXML() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024, "testwordcount", "user1"); RMApp app1 = rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -858,7 +858,7 @@ public void verifyAppInfoGeneric(RMApp app, String id, String user,
@Test @Test
public void testAppAttempts() throws JSONException, Exception { public void testAppAttempts() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024, "testwordcount", "user1"); RMApp app1 = rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testAppAttemptsHelper(app1.getApplicationId().toString(), app1, testAppAttemptsHelper(app1.getApplicationId().toString(), app1,
@ -869,7 +869,7 @@ public void testAppAttempts() throws JSONException, Exception {
@Test @Test
public void testMultipleAppAttempts() throws JSONException, Exception { public void testMultipleAppAttempts() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024, "testwordcount", "user1"); RMApp app1 = rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
int maxAppAttempts = rm.getConfig().getInt( int maxAppAttempts = rm.getConfig().getInt(
@ -895,7 +895,7 @@ public void testMultipleAppAttempts() throws JSONException, Exception {
@Test @Test
public void testAppAttemptsSlash() throws JSONException, Exception { public void testAppAttemptsSlash() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testAppAttemptsHelper(app1.getApplicationId().toString() + "/", app1, testAppAttemptsHelper(app1.getApplicationId().toString() + "/", app1,
@ -906,7 +906,7 @@ public void testAppAttemptsSlash() throws JSONException, Exception {
@Test @Test
public void testAppAttemtpsDefault() throws JSONException, Exception { public void testAppAttemtpsDefault() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024); RMApp app1 = rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
testAppAttemptsHelper(app1.getApplicationId().toString() + "/", app1, ""); testAppAttemptsHelper(app1.getApplicationId().toString() + "/", app1, "");
@ -916,7 +916,7 @@ public void testAppAttemtpsDefault() throws JSONException, Exception {
@Test @Test
public void testInvalidAppAttempts() throws JSONException, Exception { public void testInvalidAppAttempts() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024); rm.submitApp(1024);
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -952,7 +952,7 @@ public void testInvalidAppAttempts() throws JSONException, Exception {
@Test @Test
public void testNonexistAppAttempts() throws JSONException, Exception { public void testNonexistAppAttempts() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
rm.submitApp(1024, "testwordcount", "user1"); rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();
@ -1014,7 +1014,7 @@ public void testAppAttemptsHelper(String path, RMApp app, String media)
@Test @Test
public void testAppAttemptsXML() throws JSONException, Exception { public void testAppAttemptsXML() throws JSONException, Exception {
rm.start(); rm.start();
MockNM amNodeManager = rm.registerNode("amNM:1234", 2048); MockNM amNodeManager = rm.registerNode("127.0.0.1:1234", 2048);
RMApp app1 = rm.submitApp(1024, "testwordcount", "user1"); RMApp app1 = rm.submitApp(1024, "testwordcount", "user1");
amNodeManager.nodeHeartbeat(true); amNodeManager.nodeHeartbeat(true);
WebResource r = resource(); WebResource r = resource();

View File

@ -20,8 +20,6 @@
import static org.junit.Assert.fail; import static org.junit.Assert.fail;
import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException; import java.io.IOException;
import java.net.InetSocketAddress; import java.net.InetSocketAddress;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
@ -39,14 +37,11 @@
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.FileContext;
import org.apache.hadoop.fs.Path;
import org.apache.hadoop.fs.UnsupportedFileSystemException; import org.apache.hadoop.fs.UnsupportedFileSystemException;
import org.apache.hadoop.io.DataInputBuffer; import org.apache.hadoop.io.DataInputBuffer;
import org.apache.hadoop.io.Text; import org.apache.hadoop.io.Text;
import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RPC;
import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.net.NetUtils;
import org.apache.hadoop.security.AccessControlException;
import org.apache.hadoop.security.SecurityUtil; 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.security.token.Token;
@ -85,8 +80,6 @@
import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager; import org.apache.hadoop.yarn.server.resourcemanager.security.RMContainerTokenSecretManager;
import org.apache.hadoop.yarn.util.BuilderUtils; import org.apache.hadoop.yarn.util.BuilderUtils;
import org.apache.hadoop.yarn.util.Records; import org.apache.hadoop.yarn.util.Records;
import org.junit.AfterClass;
import org.junit.BeforeClass;
import org.junit.Test; import org.junit.Test;
public class TestContainerManagerSecurity { public class TestContainerManagerSecurity {
@ -94,39 +87,51 @@ public class TestContainerManagerSecurity {
static Log LOG = LogFactory.getLog(TestContainerManagerSecurity.class); static Log LOG = LogFactory.getLog(TestContainerManagerSecurity.class);
static final RecordFactory recordFactory = RecordFactoryProvider static final RecordFactory recordFactory = RecordFactoryProvider
.getRecordFactory(null); .getRecordFactory(null);
private static FileContext localFS = null;
private static final File localDir = new File("target",
TestContainerManagerSecurity.class.getName() + "-localDir")
.getAbsoluteFile();
private static MiniYARNCluster yarnCluster; private static MiniYARNCluster yarnCluster;
static final Configuration conf = new Configuration(); static final Configuration conf = new Configuration();
@BeforeClass @Test (timeout = 1000000)
public static void setup() throws AccessControlException, public void testContainerManagerWithSecurityEnabled() throws Exception {
FileNotFoundException, UnsupportedFileSystemException, IOException {
localFS = FileContext.getLocalFSFileContext();
localFS.delete(new Path(localDir.getAbsolutePath()), true);
localDir.mkdir();
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos"); "kerberos");
// Set AM expiry interval to be very long. testContainerManager();
conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 100000L); }
UserGroupInformation.setConfiguration(conf);
@Test (timeout=1000000)
public void testContainerManagerWithSecurityDisabled() throws Exception {
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"simple");
testContainerManager();
}
private void testContainerManager() throws Exception {
try {
yarnCluster = new MiniYARNCluster(TestContainerManagerSecurity.class yarnCluster = new MiniYARNCluster(TestContainerManagerSecurity.class
.getName(), 1, 1, 1); .getName(), 1, 1, 1);
conf.setLong(YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS, 100000L);
UserGroupInformation.setConfiguration(conf);
yarnCluster.init(conf); yarnCluster.init(conf);
yarnCluster.start(); yarnCluster.start();
}
@AfterClass // Testing for authenticated user
public static void teardown() { testAuthenticatedUser();
// Testing for malicious user
testMaliceUser();
// Testing for unauthorized user
testUnauthorizedUser();
} finally {
if (yarnCluster != null) {
yarnCluster.stop(); yarnCluster.stop();
yarnCluster = null;
}
}
} }
@Test private void testAuthenticatedUser() throws IOException,
public void testAuthenticatedUser() throws IOException,
InterruptedException, YarnRemoteException { InterruptedException, YarnRemoteException {
LOG.info("Running test for authenticated user"); LOG.info("Running test for authenticated user");
@ -178,8 +183,7 @@ public Void run() throws Exception {
resourceManager.getClientRMService().forceKillApplication(request); resourceManager.getClientRMService().forceKillApplication(request);
} }
@Test private void testMaliceUser() throws IOException, InterruptedException,
public void testMaliceUser() throws IOException, InterruptedException,
YarnRemoteException { YarnRemoteException {
LOG.info("Running test for malice user"); LOG.info("Running test for malice user");
@ -264,8 +268,7 @@ public Void run() {
resourceManager.getClientRMService().forceKillApplication(request); resourceManager.getClientRMService().forceKillApplication(request);
} }
@Test private void testUnauthorizedUser() throws IOException, InterruptedException,
public void testUnauthorizedUser() throws IOException, InterruptedException,
YarnRemoteException { YarnRemoteException {
LOG.info("\n\nRunning test for malice user"); LOG.info("\n\nRunning test for malice user");
@ -315,9 +318,9 @@ public ContainerManager run() {
LOG.info("Going to contact NM: unauthorized request"); LOG.info("Going to contact NM: unauthorized request");
callWithIllegalContainerID(client, tokenId); callWithIllegalContainerID(client, tokenId, allocatedContainer);
callWithIllegalResource(client, tokenId); callWithIllegalResource(client, tokenId, allocatedContainer);
callWithIllegalUserName(client, tokenId); callWithIllegalUserName(client, tokenId, allocatedContainer);
return client; return client;
} }
@ -335,10 +338,11 @@ public ContainerManager run() {
resourceManager.getRMContainerTokenSecretManager(); resourceManager.getRMContainerTokenSecretManager();
final ContainerTokenIdentifier newTokenId = final ContainerTokenIdentifier newTokenId =
new ContainerTokenIdentifier(tokenId.getContainerID(), new ContainerTokenIdentifier(tokenId.getContainerID(),
tokenId.getNmHostAddress(), "testUser", tokenId.getResource(), tokenId.getNmHostAddress(), tokenId.getApplicationSubmitter(),
tokenId.getResource(),
System.currentTimeMillis() - 1, System.currentTimeMillis() - 1,
containerTokenSecreteManager.getCurrentKey().getKeyId()); containerTokenSecreteManager.getCurrentKey().getKeyId());
byte[] passowrd = final byte[] passowrd =
containerTokenSecreteManager.createPassword( containerTokenSecreteManager.createPassword(
newTokenId); newTokenId);
// Create a valid token by using the key from the RM. // Create a valid token by using the key from the RM.
@ -357,13 +361,12 @@ public Void run() {
LOG.info("Going to contact NM with expired token"); LOG.info("Going to contact NM with expired token");
ContainerLaunchContext context = createContainerLaunchContextForTest(newTokenId); ContainerLaunchContext context = createContainerLaunchContextForTest(newTokenId);
Container container = StartContainerRequest request =
BuilderUtils.newContainer(newTokenId.getContainerID(), null, null, Records.newRecord(StartContainerRequest.class);
BuilderUtils.newResource(newTokenId.getResource().getMemory(),
newTokenId.getResource().getVirtualCores()), null, null, 0);
StartContainerRequest request = Records.newRecord(StartContainerRequest.class);
request.setContainerLaunchContext(context); request.setContainerLaunchContext(context);
request.setContainer(container); allocatedContainer.setContainerToken(BuilderUtils.newContainerToken(
allocatedContainer.getNodeId(), passowrd, newTokenId));
request.setContainer(allocatedContainer);
//Calling startContainer with an expired token. //Calling startContainer with an expired token.
try { try {
@ -447,6 +450,7 @@ private AMRMProtocol submitAndRegisterApplication(
// Ask for a container from the RM // Ask for a container from the RM
final InetSocketAddress schedulerAddr = final InetSocketAddress schedulerAddr =
resourceManager.getApplicationMasterService().getBindAddress(); resourceManager.getApplicationMasterService().getBindAddress();
if (UserGroupInformation.isSecurityEnabled()) {
ApplicationTokenIdentifier appTokenIdentifier = new ApplicationTokenIdentifier( ApplicationTokenIdentifier appTokenIdentifier = new ApplicationTokenIdentifier(
appAttempt.getAppAttemptId()); appAttempt.getAppAttemptId());
ApplicationTokenSecretManager appTokenSecretManager = ApplicationTokenSecretManager appTokenSecretManager =
@ -458,6 +462,7 @@ private AMRMProtocol submitAndRegisterApplication(
appTokenSecretManager); appTokenSecretManager);
SecurityUtil.setTokenService(appToken, schedulerAddr); SecurityUtil.setTokenService(appToken, schedulerAddr);
currentUser.addToken(appToken); currentUser.addToken(appToken);
}
AMRMProtocol scheduler = currentUser AMRMProtocol scheduler = currentUser
.doAs(new PrivilegedAction<AMRMProtocol>() { .doAs(new PrivilegedAction<AMRMProtocol>() {
@ -513,16 +518,20 @@ private Container requestAndGetContainer(AMRMProtocol scheduler,
} }
void callWithIllegalContainerID(ContainerManager client, void callWithIllegalContainerID(ContainerManager client,
ContainerTokenIdentifier tokenId) { ContainerTokenIdentifier tokenId, Container container) {
GetContainerStatusRequest request = recordFactory StartContainerRequest request = recordFactory
.newRecordInstance(GetContainerStatusRequest.class); .newRecordInstance(StartContainerRequest.class);
ContainerLaunchContext context =
createContainerLaunchContextForTest(tokenId);
ContainerId newContainerId = BuilderUtils.newContainerId(BuilderUtils ContainerId newContainerId = BuilderUtils.newContainerId(BuilderUtils
.newApplicationAttemptId(tokenId.getContainerID() .newApplicationAttemptId(tokenId.getContainerID()
.getApplicationAttemptId().getApplicationId(), 1), 42); .getApplicationAttemptId().getApplicationId(), 1), 42);
request.setContainerId(newContainerId); // Authenticated but ContainerId oldContainerId = container.getId();
// unauthorized.
try { try {
client.getContainerStatus(request); container.setId(newContainerId);
request.setContainer(container);
request.setContainerLaunchContext(context);
client.startContainer(request);
fail("Connection initiation with unauthorized " fail("Connection initiation with unauthorized "
+ "access is expected to fail."); + "access is expected to fail.");
} catch (YarnRemoteException e) { } catch (YarnRemoteException e) {
@ -534,19 +543,20 @@ void callWithIllegalContainerID(ContainerManager client,
} catch (IOException e) { } catch (IOException e) {
LOG.info("Got IOException: ",e); LOG.info("Got IOException: ",e);
fail("IOException is not expected."); fail("IOException is not expected.");
} finally {
container.setId(oldContainerId);
} }
} }
void callWithIllegalResource(ContainerManager client, void callWithIllegalResource(ContainerManager client,
ContainerTokenIdentifier tokenId) { ContainerTokenIdentifier tokenId, Container container) {
StartContainerRequest request = recordFactory StartContainerRequest request = recordFactory
.newRecordInstance(StartContainerRequest.class); .newRecordInstance(StartContainerRequest.class);
// Authenticated but unauthorized, due to wrong resource // Authenticated but unauthorized, due to wrong resource
ContainerLaunchContext context = ContainerLaunchContext context =
createContainerLaunchContextForTest(tokenId); createContainerLaunchContextForTest(tokenId);
Container container = Resource rsrc = container.getResource();
BuilderUtils.newContainer(tokenId.getContainerID(), null, null, container.setResource(BuilderUtils.newResource(2048, 1));
BuilderUtils.newResource(2048, 1), null, null, 0);
request.setContainerLaunchContext(context); request.setContainerLaunchContext(context);
request.setContainer(container); request.setContainer(container);
try { try {
@ -564,20 +574,17 @@ void callWithIllegalResource(ContainerManager client,
LOG.info("Got IOException: ",e); LOG.info("Got IOException: ",e);
fail("IOException is not expected."); fail("IOException is not expected.");
} }
container.setResource(rsrc);
} }
void callWithIllegalUserName(ContainerManager client, void callWithIllegalUserName(ContainerManager client,
ContainerTokenIdentifier tokenId) { ContainerTokenIdentifier tokenId, Container container) {
StartContainerRequest request = recordFactory StartContainerRequest request = recordFactory
.newRecordInstance(StartContainerRequest.class); .newRecordInstance(StartContainerRequest.class);
// Authenticated but unauthorized, due to wrong resource // Authenticated but unauthorized, due to wrong resource
ContainerLaunchContext context = ContainerLaunchContext context =
createContainerLaunchContextForTest(tokenId); createContainerLaunchContextForTest(tokenId);
context.setUser("Saruman"); // Set a different user-name. context.setUser("Saruman"); // Set a different user-name.
Container container =
BuilderUtils.newContainer(tokenId.getContainerID(), null, null,
BuilderUtils.newResource(tokenId.getResource().getMemory(), tokenId
.getResource().getVirtualCores()), null, null, 0);
request.setContainerLaunchContext(context); request.setContainerLaunchContext(context);
request.setContainer(container); request.setContainer(container);
try { try {
@ -601,7 +608,8 @@ private ContainerLaunchContext createContainerLaunchContextForTest(
ContainerTokenIdentifier tokenId) { ContainerTokenIdentifier tokenId) {
ContainerLaunchContext context = ContainerLaunchContext context =
BuilderUtils.newContainerLaunchContext( BuilderUtils.newContainerLaunchContext(
"testUser", new HashMap<String, LocalResource>(), tokenId.getApplicationSubmitter(),
new HashMap<String, LocalResource>(),
new HashMap<String, String>(), new ArrayList<String>(), new HashMap<String, String>(), new ArrayList<String>(),
new HashMap<String, ByteBuffer>(), null, new HashMap<String, ByteBuffer>(), null,
new HashMap<ApplicationAccessType, String>()); new HashMap<ApplicationAccessType, String>());

View File

@ -37,17 +37,25 @@
public class TestRMNMSecretKeys { public class TestRMNMSecretKeys {
@Test @Test(timeout = 1000000)
public void testNMUpdation() throws Exception { public void testNMUpdation() throws Exception {
YarnConfiguration conf = new YarnConfiguration(); YarnConfiguration conf = new YarnConfiguration();
// validating RM NM keys for Unsecured environment
validateRMNMKeyExchange(conf);
// validating RM NM keys for secured environment
conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
"kerberos"); "kerberos");
UserGroupInformation.setConfiguration(conf); UserGroupInformation.setConfiguration(conf);
validateRMNMKeyExchange(conf);
}
private void validateRMNMKeyExchange(YarnConfiguration conf) throws Exception {
// Default rolling and activation intervals are large enough, no need to // Default rolling and activation intervals are large enough, no need to
// intervene // intervene
final DrainDispatcher dispatcher = new DrainDispatcher(); final DrainDispatcher dispatcher = new DrainDispatcher();
ResourceManager rm = new ResourceManager() { ResourceManager rm = new ResourceManager() {
@Override @Override
protected void doSecureLogin() throws IOException { protected void doSecureLogin() throws IOException {
// Do nothing. // Do nothing.
@ -110,7 +118,8 @@ protected Dispatcher createDispatcher() {
dispatcher.await(); dispatcher.await();
response = nm.nodeHeartbeat(true); response = nm.nodeHeartbeat(true);
Assert.assertNull( Assert
.assertNull(
"Even second heartbeat after activation shouldn't get any key updates!", "Even second heartbeat after activation shouldn't get any key updates!",
response.getMasterKey()); response.getMasterKey());
dispatcher.await(); dispatcher.await();