|
@@ -32,9 +32,9 @@ import org.apache.commons.logging.LogFactory;
|
|
|
import org.apache.hadoop.conf.Configuration;
|
|
|
import org.apache.hadoop.io.DataInputByteBuffer;
|
|
|
import org.apache.hadoop.io.DataOutputBuffer;
|
|
|
-import org.apache.hadoop.io.Text;
|
|
|
import org.apache.hadoop.net.NetUtils;
|
|
|
import org.apache.hadoop.security.Credentials;
|
|
|
+import org.apache.hadoop.security.SecurityUtil;
|
|
|
import org.apache.hadoop.security.UserGroupInformation;
|
|
|
import org.apache.hadoop.security.token.Token;
|
|
|
import org.apache.hadoop.util.StringUtils;
|
|
@@ -46,7 +46,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
|
|
|
import org.apache.hadoop.yarn.api.records.Container;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerId;
|
|
|
import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
|
|
|
-import org.apache.hadoop.yarn.api.records.ContainerToken;
|
|
|
+import org.apache.hadoop.yarn.api.records.NodeId;
|
|
|
import org.apache.hadoop.yarn.conf.YarnConfiguration;
|
|
|
import org.apache.hadoop.yarn.event.EventHandler;
|
|
|
import org.apache.hadoop.yarn.factories.RecordFactory;
|
|
@@ -61,6 +61,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttempt;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEvent;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.RMAppAttemptEventType;
|
|
|
import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.event.RMAppAttemptLaunchFailedEvent;
|
|
|
+import org.apache.hadoop.yarn.util.ProtoUtils;
|
|
|
|
|
|
/**
|
|
|
* The launch of the AM itself.
|
|
@@ -131,27 +132,25 @@ public class AMLauncher implements Runnable {
|
|
|
|
|
|
Container container = application.getMasterContainer();
|
|
|
|
|
|
- final String containerManagerBindAddress = container.getNodeId().toString();
|
|
|
+ final NodeId node = container.getNodeId();
|
|
|
+ final InetSocketAddress containerManagerBindAddress =
|
|
|
+ NetUtils.createSocketAddrForHost(node.getHost(), node.getPort());
|
|
|
|
|
|
final YarnRPC rpc = YarnRPC.create(conf); // TODO: Don't create again and again.
|
|
|
|
|
|
UserGroupInformation currentUser = UserGroupInformation
|
|
|
.createRemoteUser(containerId.toString());
|
|
|
if (UserGroupInformation.isSecurityEnabled()) {
|
|
|
- ContainerToken containerToken = container.getContainerToken();
|
|
|
Token<ContainerTokenIdentifier> token =
|
|
|
- new Token<ContainerTokenIdentifier>(
|
|
|
- containerToken.getIdentifier().array(),
|
|
|
- containerToken.getPassword().array(), new Text(
|
|
|
- containerToken.getKind()), new Text(
|
|
|
- containerToken.getService()));
|
|
|
+ ProtoUtils.convertFromProtoFormat(container.getContainerToken(),
|
|
|
+ containerManagerBindAddress);
|
|
|
currentUser.addToken(token);
|
|
|
}
|
|
|
return currentUser.doAs(new PrivilegedAction<ContainerManager>() {
|
|
|
@Override
|
|
|
public ContainerManager run() {
|
|
|
return (ContainerManager) rpc.getProxy(ContainerManager.class,
|
|
|
- NetUtils.createSocketAddr(containerManagerBindAddress), conf);
|
|
|
+ containerManagerBindAddress, conf);
|
|
|
}
|
|
|
});
|
|
|
}
|
|
@@ -218,22 +217,21 @@ public class AMLauncher implements Runnable {
|
|
|
Token<ApplicationTokenIdentifier> token =
|
|
|
new Token<ApplicationTokenIdentifier>(id,
|
|
|
this.rmContext.getApplicationTokenSecretManager());
|
|
|
- InetSocketAddress unresolvedAddr = conf.getSocketAddr(
|
|
|
+ InetSocketAddress serviceAddr = conf.getSocketAddr(
|
|
|
YarnConfiguration.RM_SCHEDULER_ADDRESS,
|
|
|
YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS,
|
|
|
YarnConfiguration.DEFAULT_RM_SCHEDULER_PORT);
|
|
|
- String resolvedAddr =
|
|
|
- unresolvedAddr.getAddress().getHostAddress() + ":"
|
|
|
- + unresolvedAddr.getPort();
|
|
|
- token.setService(new Text(resolvedAddr));
|
|
|
+ // normally the client should set the service after acquiring the token,
|
|
|
+ // but this token is directly provided to the tasks
|
|
|
+ SecurityUtil.setTokenService(token, serviceAddr);
|
|
|
String appMasterTokenEncoded = token.encodeToUrlString();
|
|
|
- LOG.debug("Putting appMaster token in env : " + appMasterTokenEncoded);
|
|
|
+ LOG.debug("Putting appMaster token in env : " + token);
|
|
|
environment.put(
|
|
|
ApplicationConstants.APPLICATION_MASTER_TOKEN_ENV_NAME,
|
|
|
appMasterTokenEncoded);
|
|
|
|
|
|
// Add the RM token
|
|
|
- credentials.addToken(new Text(resolvedAddr), token);
|
|
|
+ credentials.addToken(token.getService(), token);
|
|
|
DataOutputBuffer dob = new DataOutputBuffer();
|
|
|
credentials.writeTokenStorageToStream(dob);
|
|
|
container.setContainerTokens(
|
|
@@ -245,7 +243,6 @@ public class AMLauncher implements Runnable {
|
|
|
this.clientToAMSecretManager.getMasterKey(identifier);
|
|
|
String encoded =
|
|
|
Base64.encodeBase64URLSafeString(clientSecretKey.getEncoded());
|
|
|
- LOG.debug("The encoded client secret-key to be put in env : " + encoded);
|
|
|
environment.put(
|
|
|
ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME,
|
|
|
encoded);
|