Browse Source

YARN-610. ClientToken is no longer set in the environment of the Containers. Contributed by Omkar Vinit Joshi.

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/trunk@1493968 13f79535-47bb-0310-9956-ffa450edef68
Vinod Kumar Vavilapalli 12 years ago
parent
commit
5d1b453b85
40 changed files with 341 additions and 310 deletions
  1. 3 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java
  2. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java
  3. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRClientSecurityInfo.java
  4. 1 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java
  5. 9 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java
  6. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java
  7. 6 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
  8. 7 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
  9. 5 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java
  10. 3 0
      hadoop-yarn-project/CHANGES.txt
  11. 0 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java
  12. 15 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java
  13. 20 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java
  14. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java
  15. 15 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java
  16. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  17. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  18. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java
  19. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java
  20. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/BaseClientToAMTokenSecretManager.java
  21. 19 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenIdentifier.java
  22. 11 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenSecretManager.java
  23. 54 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenSelector.java
  24. 0 58
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientTokenSelector.java
  25. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier
  26. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
  27. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java
  28. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java
  29. 2 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  30. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java
  31. 12 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java
  32. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java
  33. 13 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java
  34. 4 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java
  35. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  36. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java
  37. 8 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  38. 0 35
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java
  39. 16 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStore.java
  40. 62 64
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java

+ 3 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/AppContext.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.mapreduce.v2.app.job.Job;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
 
 
@@ -58,4 +59,6 @@ public interface AppContext {
   ClusterInfo getClusterInfo();
   
   Set<String> getBlacklistedNodes();
+  
+  ClientToAMTokenSecretManager getClientToAMTokenSecretManager();
 }

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRAppMaster.java

@@ -128,6 +128,7 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.SystemClock;
@@ -884,9 +885,12 @@ public class MRAppMaster extends CompositeService {
     private final Map<JobId, Job> jobs = new ConcurrentHashMap<JobId, Job>();
     private final Configuration conf;
     private final ClusterInfo clusterInfo = new ClusterInfo();
+    private final ClientToAMTokenSecretManager clientToAMTokenSecretManager;
 
     public RunningAppContext(Configuration config) {
       this.conf = config;
+      this.clientToAMTokenSecretManager =
+          new ClientToAMTokenSecretManager(appAttemptID, null);
     }
 
     @Override
@@ -943,6 +947,11 @@ public class MRAppMaster extends CompositeService {
     public Set<String> getBlacklistedNodes() {
       return ((RMContainerRequestor) containerAllocator).getBlacklistedNodes();
     }
+    
+    @Override
+    public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
+      return clientToAMTokenSecretManager;
+    }
   }
 
   @SuppressWarnings("unchecked")

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/MRClientSecurityInfo.java

@@ -27,7 +27,7 @@ import org.apache.hadoop.security.SecurityInfo;
 import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.security.token.TokenSelector;
-import org.apache.hadoop.yarn.security.client.ClientTokenSelector;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSelector;
 
 public class MRClientSecurityInfo extends SecurityInfo {
 
@@ -51,7 +51,7 @@ public class MRClientSecurityInfo extends SecurityInfo {
       @Override
       public Class<? extends TokenSelector<? extends TokenIdentifier>>
           value() {
-        return ClientTokenSelector.class;
+        return ClientToAMTokenSelector.class;
       }
     };
   }

+ 1 - 15
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/client/MRClientService.java

@@ -23,7 +23,6 @@ import java.net.InetSocketAddress;
 import java.util.Arrays;
 import java.util.Collection;
 
-import org.apache.commons.codec.binary.Base64;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -79,14 +78,11 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app.security.authorize.MRAMPolicyProvider;
 import org.apache.hadoop.mapreduce.v2.app.webapp.AMWebApp;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.authorize.PolicyProvider;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.webapp.WebApp;
 import org.apache.hadoop.yarn.webapp.WebApps;
 
@@ -117,19 +113,9 @@ public class MRClientService extends AbstractService
     YarnRPC rpc = YarnRPC.create(conf);
     InetSocketAddress address = new InetSocketAddress(0);
 
-    ClientToAMTokenSecretManager secretManager = null;
-    if (UserGroupInformation.isSecurityEnabled()) {
-      String secretKeyStr =
-          System
-              .getenv(ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME);
-      byte[] bytes = Base64.decodeBase64(secretKeyStr);
-      secretManager =
-          new ClientToAMTokenSecretManager(
-            this.appContext.getApplicationAttemptId(), bytes);
-    }
     server =
         rpc.getServer(MRClientProtocol.class, protocolHandler, address,
-            conf, secretManager,
+            conf, appContext.getClientToAMTokenSecretManager(),
             conf.getInt(MRJobConfig.MR_AM_JOB_CLIENT_THREAD_COUNT, 
                 MRJobConfig.DEFAULT_MR_AM_JOB_CLIENT_THREAD_COUNT),
                 MRJobConfig.MR_AM_JOB_CLIENT_PORT_RANGE);

+ 9 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/rm/RMCommunicator.java

@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.app.rm;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
 import java.util.Map;
 import java.util.concurrent.ConcurrentLinkedQueue;
@@ -154,6 +155,9 @@ public abstract class RMCommunicator extends AbstractService
       maxContainerCapability = response.getMaximumResourceCapability();
       this.context.getClusterInfo().setMaxContainerCapability(
           maxContainerCapability);
+      if (UserGroupInformation.isSecurityEnabled()) {
+        setClientToAMToken(response.getClientToAMTokenMasterKey());        
+      }
       this.applicationACLs = response.getApplicationACLs();
       LOG.info("maxContainerCapability: " + maxContainerCapability.getMemory());
     } catch (Exception are) {
@@ -162,6 +166,11 @@ public abstract class RMCommunicator extends AbstractService
     }
   }
 
+  private void setClientToAMToken(ByteBuffer clientToAMTokenMasterKey) {
+    byte[] key = clientToAMTokenMasterKey.array();
+    context.getClientToAMTokenSecretManager().setMasterKey(key);
+  }
+
   protected void unregister() {
     try {
       FinalApplicationStatus finishState = FinalApplicationStatus.UNDEFINED;

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockAppContext.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 
 import com.google.common.collect.Maps;
 
@@ -125,4 +126,8 @@ public class MockAppContext implements AppContext {
     this.blacklistedNodes = blacklistedNodes;
   }
 
+  public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
+    // Not implemented
+    return null;
+  }
 }

+ 6 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java

@@ -72,6 +72,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
 import org.junit.Assert;
@@ -855,5 +856,10 @@ public class TestRuntimeEstimators {
     public Set<String> getBlacklistedNodes() {
       return null;
     }
+    
+    public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
+      // Not Implemented
+      return null;
+    }
   }
 }

+ 7 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java

@@ -47,6 +47,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
 import org.apache.hadoop.yarn.util.Clock;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
@@ -306,6 +307,12 @@ public class JobHistory extends AbstractService implements HistoryContext {
   // TODO AppContext - Not Required
   @Override
   public Set<String> getBlacklistedNodes() {
+    // Not Implemented
+    return null;
+  }
+  @Override
+  public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
+    // Not implemented.
     return null;
   }
 }

+ 5 - 4
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/ClientServiceDelegate.java

@@ -74,7 +74,7 @@ import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
-import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 
 public class ClientServiceDelegate {
@@ -180,9 +180,10 @@ public class ClientServiceDelegate {
           serviceAddr = NetUtils.createSocketAddrForHost(
               application.getHost(), application.getRpcPort());
           if (UserGroupInformation.isSecurityEnabled()) {
-            org.apache.hadoop.yarn.api.records.Token clientToken = application.getClientToken();
-            Token<ClientTokenIdentifier> token =
-                ProtoUtils.convertFromProtoFormat(clientToken, serviceAddr);
+            org.apache.hadoop.yarn.api.records.Token clientToAMToken =
+                application.getClientToAMToken();
+            Token<ClientToAMTokenIdentifier> token =
+                ProtoUtils.convertFromProtoFormat(clientToAMToken, serviceAddr);
             newUgi.addToken(token);
           }
           LOG.debug("Connecting to " + serviceAddr);

+ 3 - 0
hadoop-yarn-project/CHANGES.txt

@@ -183,6 +183,9 @@ Release 2.1.0-beta - UNRELEASED
     YARN-822. Renamed ApplicationToken to be AMRMToken, and similarly the
     corresponding TokenSelector and SecretManager. (Omkar Vinit Joshi via vinodkv)
 
+    YARN-610. ClientToken is no longer set in the environment of the Containers.
+    (Omkar Vinit Joshi via vinodkv)
+
   NEW FEATURES
 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 

+ 0 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ApplicationConstants.java

@@ -28,10 +28,6 @@ import org.apache.hadoop.util.Shell;
  * TODO: Investigate the semantics and security of each cross-boundary refs.
  */
 public interface ApplicationConstants {
-
-  // TODO: They say tokens via env isn't good.
-  public static final String APPLICATION_CLIENT_SECRET_ENV_NAME =
-    "AppClientSecretEnv";
   
   /**
    * The environment variable for APP_SUBMIT_TIME. Set in AppMaster environment

+ 15 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/RegisterApplicationMasterResponse.java

@@ -18,6 +18,7 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.nio.ByteBuffer;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -84,4 +85,18 @@ public abstract class RegisterApplicationMasterResponse {
   @Private
   @Unstable
   public abstract void setApplicationACLs(Map<ApplicationAccessType, String> acls);
+  
+  /**
+   * Set ClientToAMToken master key.
+   */
+  @Public
+  @Stable
+  public abstract void setClientToAMTokenMasterKey(ByteBuffer key);
+  
+  /**
+   * Get ClientToAMToken master key.
+   */
+  @Public
+  @Stable
+  public abstract ByteBuffer getClientToAMTokenMasterKey();
 }

+ 20 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/RegisterApplicationMasterResponsePBImpl.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -34,6 +35,8 @@ import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterR
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.RegisterApplicationMasterResponseProtoOrBuilder;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 
+import com.google.protobuf.ByteString;
+
 
 public class RegisterApplicationMasterResponsePBImpl extends
     RegisterApplicationMasterResponse {
@@ -200,7 +203,23 @@ public class RegisterApplicationMasterResponsePBImpl extends
     this.applicationACLS.clear();
     this.applicationACLS.putAll(appACLs);
   }
-
+  
+  @Override
+  public void setClientToAMTokenMasterKey(ByteBuffer key) {
+    if (key == null) {
+      return;
+    }
+    maybeInitBuilder();
+    builder.setClientToAmTokenMasterKey(ByteString.copyFrom(key));
+  }
+  
+  @Override
+  public ByteBuffer getClientToAMTokenMasterKey() {
+    ByteBuffer key =
+        ByteBuffer.wrap(builder.getClientToAmTokenMasterKey().toByteArray());
+    return key;
+  }
+  
   private Resource convertFromProtoFormat(ResourceProto resource) {
     return new ResourcePBImpl(resource);
   }

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationReport.java

@@ -54,7 +54,7 @@ public abstract class ApplicationReport {
   @Stable
   public static ApplicationReport newInstance(ApplicationId applicationId,
       ApplicationAttemptId applicationAttemptId, String user, String queue,
-      String name, String host, int rpcPort, Token clientToken,
+      String name, String host, int rpcPort, Token clientToAMToken,
       YarnApplicationState state, String diagnostics, String url,
       long startTime, long finishTime, FinalApplicationStatus finalStatus,
       ApplicationResourceUsageReport appResources, String origTrackingUrl,
@@ -67,7 +67,7 @@ public abstract class ApplicationReport {
     report.setName(name);
     report.setHost(host);
     report.setRpcPort(rpcPort);
-    report.setClientToken(clientToken);
+    report.setClientToAMToken(clientToAMToken);
     report.setYarnApplicationState(state);
     report.setDiagnostics(diagnostics);
     report.setTrackingUrl(url);
@@ -172,13 +172,13 @@ public abstract class ApplicationReport {
    * Get the <em>client token</em> for communicating with the
    * <code>ApplicationMaster</code>.
    * <p>
-   * <code>ClientToken</code> is the security token used by the AMs to verify
+   * <em>ClientToAMToken</em> is the security token used by the AMs to verify
    * authenticity of any <code>client</code>.
    * </p>
    *
    * <p>
    * The <code>ResourceManager</code>, provides a secure token (via
-   * {@link ApplicationReport#getClientToken()}) which is verified by the
+   * {@link ApplicationReport#getClientToAMToken()}) which is verified by the
    * ApplicationMaster when the client directly talks to an AM.
    * </p>
    * @return <em>client token</em> for communicating with the
@@ -186,11 +186,11 @@ public abstract class ApplicationReport {
    */
   @Public
   @Stable
-  public abstract Token getClientToken();
+  public abstract Token getClientToAMToken();
 
   @Private
   @Unstable
-  public abstract void setClientToken(Token clientToken);
+  public abstract void setClientToAMToken(Token clientToAMToken);
 
   /**
    * Get the <code>YarnApplicationState</code> of the application.

+ 15 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationReportPBImpl.java

@@ -42,7 +42,7 @@ public class ApplicationReportPBImpl extends ApplicationReport {
 
   private ApplicationId applicationId;
   private ApplicationAttemptId currentApplicationAttemptId;
-  private Token clientToken = null;
+  private Token clientToAMToken = null;
 
   public ApplicationReportPBImpl() {
     builder = ApplicationReportProto.newBuilder();
@@ -160,16 +160,16 @@ public class ApplicationReportPBImpl extends ApplicationReport {
   }
 
   @Override
-  public Token getClientToken() {
+  public Token getClientToAMToken() {
     ApplicationReportProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.clientToken != null) {
-      return this.clientToken;
+    if (this.clientToAMToken != null) {
+      return this.clientToAMToken;
     }
-    if (!p.hasClientToken()) {
+    if (!p.hasClientToAmToken()) {
       return null;
     }
-    this.clientToken = convertFromProtoFormat(p.getClientToken());
-    return this.clientToken;
+    this.clientToAMToken = convertFromProtoFormat(p.getClientToAmToken());
+    return this.clientToAMToken;
   }
 
   @Override
@@ -309,11 +309,11 @@ public class ApplicationReportPBImpl extends ApplicationReport {
   }
 
   @Override
-  public void setClientToken(Token clientToken) {
+  public void setClientToAMToken(Token clientToAMToken) {
     maybeInitBuilder();
-    if (clientToken == null) 
-      builder.clearClientToken();
-    this.clientToken = clientToken;
+    if (clientToAMToken == null) 
+      builder.clearClientToAmToken();
+    this.clientToAMToken = clientToAMToken;
   }
 
   @Override
@@ -412,10 +412,10 @@ public class ApplicationReportPBImpl extends ApplicationReport {
             builder.getCurrentApplicationAttemptId())) {
       builder.setCurrentApplicationAttemptId(convertToProtoFormat(this.currentApplicationAttemptId));
     }
-    if (this.clientToken != null
-        && !((TokenPBImpl) this.clientToken).getProto().equals(
-            builder.getClientToken())) {
-      builder.setClientToken(convertToProtoFormat(this.clientToken));
+    if (this.clientToAMToken != null
+        && !((TokenPBImpl) this.clientToAMToken).getProto().equals(
+            builder.getClientToAmToken())) {
+      builder.setClientToAmToken(convertToProtoFormat(this.clientToAMToken));
     }
   }
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto

@@ -131,7 +131,7 @@ message ApplicationReportProto {
   optional string name = 4;
   optional string host = 5;
   optional int32 rpc_port = 6;
-  optional hadoop.common.TokenProto client_token = 7;
+  optional hadoop.common.TokenProto client_to_am_token = 7;
   optional YarnApplicationStateProto yarn_application_state = 8;
   optional string trackingUrl = 9;
   optional string diagnostics = 10 [default = "N/A"];

+ 2 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto

@@ -36,7 +36,8 @@ message RegisterApplicationMasterRequestProto {
 
 message RegisterApplicationMasterResponseProto {
   optional ResourceProto maximumCapability = 1;
-  repeated ApplicationACLMapProto application_ACLs = 2;
+  optional bytes client_to_am_token_master_key = 2;
+  repeated ApplicationACLMapProto application_ACLs = 3;
 }
 
 message FinishApplicationMasterRequestProto {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/Client.java

@@ -601,7 +601,7 @@ public class Client {
 
       LOG.info("Got application report from ASM for"
           + ", appId=" + appId.getId()
-          + ", clientToken=" + report.getClientToken()
+          + ", clientToAMToken=" + report.getClientToAMToken()
           + ", appDiagnostics=" + report.getDiagnostics()
           + ", appMasterHost=" + report.getHost()
           + ", appQueue=" + report.getQueue()

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java

@@ -388,8 +388,8 @@ public class UnmanagedAMLauncher {
 
       LOG.info("Got application report from ASM for" + ", appId="
           + appId.getId() + ", appAttemptId="
-          + report.getCurrentApplicationAttemptId() + ", clientToken="
-          + report.getClientToken() + ", appDiagnostics="
+          + report.getCurrentApplicationAttemptId() + ", clientToAMToken="
+          + report.getClientToAMToken() + ", appDiagnostics="
           + report.getDiagnostics() + ", appMasterHost=" + report.getHost()
           + ", appQueue=" + report.getQueue() + ", appMasterRpcPort="
           + report.getRpcPort() + ", appStartTime=" + report.getStartTime()

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/BaseClientToAMTokenSecretManager.java

@@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 @Public
 @Evolving
 public abstract class BaseClientToAMTokenSecretManager extends
-    SecretManager<ClientTokenIdentifier> {
+    SecretManager<ClientToAMTokenIdentifier> {
 
   @Private
   public abstract SecretKey getMasterKey(
@@ -45,14 +45,14 @@ public abstract class BaseClientToAMTokenSecretManager extends
   @Private
   @Override
   public synchronized byte[] createPassword(
-      ClientTokenIdentifier identifier) {
+      ClientToAMTokenIdentifier identifier) {
     return createPassword(identifier.getBytes(),
       getMasterKey(identifier.getApplicationAttemptID()));
   }
 
   @Private
   @Override
-  public byte[] retrievePassword(ClientTokenIdentifier identifier)
+  public byte[] retrievePassword(ClientToAMTokenIdentifier identifier)
       throws SecretManager.InvalidToken {
     SecretKey masterKey = getMasterKey(identifier.getApplicationAttemptID());
     if (masterKey == null) {
@@ -63,8 +63,8 @@ public abstract class BaseClientToAMTokenSecretManager extends
 
   @Private
   @Override
-  public ClientTokenIdentifier createIdentifier() {
-    return new ClientTokenIdentifier();
+  public ClientToAMTokenIdentifier createIdentifier() {
+    return new ClientToAMTokenIdentifier();
   }
 
 }

+ 19 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientTokenIdentifier.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenIdentifier.java

@@ -1,20 +1,20 @@
 /**
-* 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.
-*/
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
 
 package org.apache.hadoop.yarn.security.client;
 
@@ -34,7 +34,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 
 @Public
 @Evolving
-public class ClientTokenIdentifier extends TokenIdentifier {
+public class ClientToAMTokenIdentifier extends TokenIdentifier {
 
   public static final Text KIND_NAME = new Text("YARN_CLIENT_TOKEN");
 
@@ -43,10 +43,10 @@ public class ClientTokenIdentifier extends TokenIdentifier {
   // TODO: Add more information in the tokenID such that it is not
   // transferrable, more secure etc.
 
-  public ClientTokenIdentifier() {
+  public ClientToAMTokenIdentifier() {
   }
 
-  public ClientTokenIdentifier(ApplicationAttemptId id) {
+  public ClientToAMTokenIdentifier(ApplicationAttemptId id) {
     this();
     this.applicationAttemptId = id;
   }

+ 11 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenSecretManager.java

@@ -38,12 +38,17 @@ public class ClientToAMTokenSecretManager extends
     BaseClientToAMTokenSecretManager {
 
   // Only one client-token and one master-key for AM
-  private final SecretKey masterKey;
+  private SecretKey masterKey;
 
   public ClientToAMTokenSecretManager(
-      ApplicationAttemptId applicationAttemptID, byte[] secretKeyBytes) {
+      ApplicationAttemptId applicationAttemptID, byte[] key) {
     super();
-    this.masterKey = SecretManager.createSecretKey(secretKeyBytes);
+    if (key !=  null) {
+      this.masterKey = SecretManager.createSecretKey(key);
+    } else {
+      this.masterKey = null;
+    }
+    
   }
 
   @Override
@@ -52,4 +57,7 @@ public class ClientToAMTokenSecretManager extends
     return this.masterKey;
   }
 
+  public void setMasterKey(byte[] key) {
+    this.masterKey = SecretManager.createSecretKey(key);
+  }
 }

+ 54 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientToAMTokenSelector.java

@@ -0,0 +1,54 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.security.client;
+
+import java.util.Collection;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.security.token.TokenSelector;
+
+public class ClientToAMTokenSelector implements
+    TokenSelector<ClientToAMTokenIdentifier> {
+
+  private static final Log LOG = LogFactory
+      .getLog(ClientToAMTokenSelector.class);
+
+  @SuppressWarnings("unchecked")
+  public Token<ClientToAMTokenIdentifier> selectToken(Text service,
+      Collection<Token<? extends TokenIdentifier>> tokens) {
+    if (service == null) {
+      return null;
+    }
+    LOG.debug("Looking for a token with service " + service.toString());
+    for (Token<? extends TokenIdentifier> token : tokens) {
+      LOG.debug("Token kind is " + token.getKind().toString()
+          + " and the token's service name is " + token.getService());
+      if (ClientToAMTokenIdentifier.KIND_NAME.equals(token.getKind())
+          && service.equals(token.getService())) {
+        return (Token<ClientToAMTokenIdentifier>) token;
+      }
+    }
+    return null;
+  }
+
+}

+ 0 - 58
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/security/client/ClientTokenSelector.java

@@ -1,58 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.security.client;
-
-import java.util.Collection;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
-import org.apache.hadoop.security.token.TokenSelector;
-
-@Public
-@Stable
-public class ClientTokenSelector implements
-    TokenSelector<ClientTokenIdentifier> {
-
-  private static final Log LOG = LogFactory
-      .getLog(ClientTokenSelector.class);
-
-  @SuppressWarnings("unchecked")
-  public Token<ClientTokenIdentifier> selectToken(Text service,
-      Collection<Token<? extends TokenIdentifier>> tokens) {
-    if (service == null) {
-      return null;
-    }
-    LOG.debug("Looking for a token with service " + service.toString());
-    for (Token<? extends TokenIdentifier> token : tokens) {
-      LOG.debug("Token kind is " + token.getKind().toString()
-          + " and the token's service name is " + token.getService());
-      if (ClientTokenIdentifier.KIND_NAME.equals(token.getKind())
-          && service.equals(token.getService())) {
-        return (Token<ClientTokenIdentifier>) token;
-      }
-    }
-    return null;
-  }
-
-}

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier

@@ -13,5 +13,5 @@
 #
 org.apache.hadoop.yarn.security.ContainerTokenIdentifier
 org.apache.hadoop.yarn.security.AMRMTokenIdentifier
-org.apache.hadoop.yarn.security.client.ClientTokenIdentifier
+org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier
 org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier

+ 3 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java

@@ -61,7 +61,6 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
-import org.apache.hadoop.yarn.server.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 
 /**
@@ -224,7 +223,7 @@ public class BuilderUtils {
     return newToken(Token.class, identifier, kind, password, service);
   }
 
-  public static Token newClientToken(byte[] identifier, String kind,
+  public static Token newClientToAMToken(byte[] identifier, String kind,
       byte[] password, String service) {
     return newToken(Token.class, identifier, kind, password, service);
   }
@@ -299,7 +298,7 @@ public class BuilderUtils {
   public static ApplicationReport newApplicationReport(
       ApplicationId applicationId, ApplicationAttemptId applicationAttemptId,
       String user, String queue, String name, String host, int rpcPort,
-      Token clientToken, YarnApplicationState state, String diagnostics,
+      Token clientToAMToken, YarnApplicationState state, String diagnostics,
       String url, long startTime, long finishTime,
       FinalApplicationStatus finalStatus,
       ApplicationResourceUsageReport appResources, String origTrackingUrl,
@@ -313,7 +312,7 @@ public class BuilderUtils {
     report.setName(name);
     report.setHost(host);
     report.setRpcPort(rpcPort);
-    report.setClientToken(clientToken);
+    report.setClientToAMToken(clientToAMToken);
     report.setYarnApplicationState(state);
     report.setDiagnostics(diagnostics);
     report.setTrackingUrl(url);

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ApplicationMasterService.java

@@ -214,6 +214,12 @@ public class ApplicationMasterService extends AbstractService implements
           .getMaximumResourceCapability());
       response.setApplicationACLs(app.getRMAppAttempt(applicationAttemptId)
           .getSubmissionContext().getAMContainerSpec().getApplicationACLs());
+      if (UserGroupInformation.isSecurityEnabled()) {
+        LOG.info("Setting client token master key");
+        response.setClientToAMTokenMasterKey(java.nio.ByteBuffer.wrap(rmContext
+            .getClientToAMTokenSecretManager()
+            .getMasterKey(applicationAttemptId).getEncoded()));        
+      }
       return response;
     }
   }

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMContextImpl.java

@@ -71,7 +71,7 @@ public class RMContextImpl implements RMContext {
       AMRMTokenSecretManager appTokenSecretManager,
       RMContainerTokenSecretManager containerTokenSecretManager,
       NMTokenSecretManagerInRM nmTokenSecretManager,
-      ClientToAMTokenSecretManagerInRM clientTokenSecretManager) {
+      ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager) {
     this.rmDispatcher = rmDispatcher;
     this.stateStore = store;
     this.containerAllocationExpirer = containerAllocationExpirer;
@@ -81,7 +81,7 @@ public class RMContextImpl implements RMContext {
     this.appTokenSecretManager = appTokenSecretManager;
     this.containerTokenSecretManager = containerTokenSecretManager;
     this.nmTokenSecretManager = nmTokenSecretManager;
-    this.clientToAMTokenSecretManager = clientTokenSecretManager;
+    this.clientToAMTokenSecretManager = clientToAMTokenSecretManager;
   }
 
   @VisibleForTesting
@@ -94,11 +94,11 @@ public class RMContextImpl implements RMContext {
       AMRMTokenSecretManager appTokenSecretManager,
       RMContainerTokenSecretManager containerTokenSecretManager,
       NMTokenSecretManagerInRM nmTokenSecretManager,
-      ClientToAMTokenSecretManagerInRM clientTokenSecretManager) {
+      ClientToAMTokenSecretManagerInRM clientToAMTokenSecretManager) {
     this(rmDispatcher, null, containerAllocationExpirer, amLivelinessMonitor, 
           amFinishingMonitor, tokenRenewer, appTokenSecretManager, 
           containerTokenSecretManager, nmTokenSecretManager,
-          clientTokenSecretManager);
+          clientToAMTokenSecretManager);
     RMStateStore nullStore = new NullRMStateStore();
     nullStore.setDispatcher(rmDispatcher);
     try {

+ 2 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java

@@ -165,12 +165,12 @@ public class AMLauncher implements Runnable {
             new String[0])));
     
     // Finalize the container
-    setupTokensAndEnv(container, containerID);
+    setupTokens(container, containerID);
     
     return container;
   }
 
-  private void setupTokensAndEnv(
+  private void setupTokens(
       ContainerLaunchContext container, ContainerId containerID)
       throws IOException {
     Map<String, String> environment = container.getEnvironment();
@@ -210,15 +210,6 @@ public class AMLauncher implements Runnable {
       credentials.writeTokenStorageToStream(dob);
       container.setTokens(ByteBuffer.wrap(dob.getData(), 0,
         dob.getLength()));
-
-      SecretKey clientSecretKey =
-          this.rmContext.getClientToAMTokenSecretManager().getMasterKey(
-            application.getAppAttemptId());
-      String encoded =
-          Base64.encodeBase64URLSafeString(clientSecretKey.getEncoded());
-      environment.put(
-          ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME, 
-          encoded);
     }
   }
   

+ 5 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/RMStateStore.java

@@ -42,7 +42,7 @@ import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationAttemptStateDataPBImpl;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.records.impl.pb.ApplicationStateDataPBImpl;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
@@ -382,9 +382,10 @@ public abstract class RMStateStore {
     if(appToken != null){
       credentials.addToken(appToken.getService(), appToken);
     }
-    Token<ClientTokenIdentifier> clientToken = appAttempt.getClientToken();
-    if(clientToken != null){
-      credentials.addToken(clientToken.getService(), clientToken);
+    Token<ClientToAMTokenIdentifier> clientToAMToken =
+        appAttempt.getClientToAMToken();
+    if(clientToAMToken != null){
+      credentials.addToken(clientToAMToken.getService(), clientToAMToken);
     }
     return credentials;
   }

+ 12 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/RMAppImpl.java

@@ -47,7 +47,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.RMAppManagerEventType;
@@ -430,7 +430,7 @@ public class RMAppImpl implements RMApp, Recoverable {
 
     try {
       ApplicationAttemptId currentApplicationAttemptId = null;
-      org.apache.hadoop.yarn.api.records.Token clientToken = null;
+      org.apache.hadoop.yarn.api.records.Token clientToAMToken = null;
       String trackingUrl = UNAVAILABLE;
       String host = UNAVAILABLE;
       String origTrackingUrl = UNAVAILABLE;
@@ -445,13 +445,15 @@ public class RMAppImpl implements RMApp, Recoverable {
           currentApplicationAttemptId = this.currentAttempt.getAppAttemptId();
           trackingUrl = this.currentAttempt.getTrackingUrl();
           origTrackingUrl = this.currentAttempt.getOriginalTrackingUrl();
-          Token<ClientTokenIdentifier> attemptClientToken =
-              this.currentAttempt.getClientToken();
-          if (attemptClientToken != null) {
-            clientToken =
-                BuilderUtils.newClientToken(attemptClientToken.getIdentifier(),
-                  attemptClientToken.getKind().toString(), attemptClientToken
-                    .getPassword(), attemptClientToken.getService().toString());
+          Token<ClientToAMTokenIdentifier> attemptClientToAMToken =
+              this.currentAttempt.getClientToAMToken();
+          if (attemptClientToAMToken != null) {
+            clientToAMToken =
+                BuilderUtils.newClientToAMToken(
+                    attemptClientToAMToken.getIdentifier(),
+                    attemptClientToAMToken.getKind().toString(),
+                    attemptClientToAMToken.getPassword(),
+                    attemptClientToAMToken.getService().toString());
           }
           host = this.currentAttempt.getHost();
           rpcPort = this.currentAttempt.getRpcPort();
@@ -469,7 +471,7 @@ public class RMAppImpl implements RMApp, Recoverable {
 
       return BuilderUtils.newApplicationReport(this.applicationId,
           currentApplicationAttemptId, this.user, this.queue,
-          this.name, host, rpcPort, clientToken,
+          this.name, host, rpcPort, clientToAMToken,
           createApplicationState(this.stateMachine.getCurrentState()), diags,
           trackingUrl, this.startTime, this.finishTime, finishState,
           appUsageReport, origTrackingUrl, progress, this.applicationType);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttempt.java

@@ -32,7 +32,7 @@ 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.security.AMRMTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.rmapp.RMApp;
 
 /**
@@ -94,7 +94,7 @@ public interface RMAppAttempt extends EventHandler<RMAppAttemptEvent> {
    * The token required by the clients to talk to the application attempt
    * @return the token required by the clients to talk to the application attempt
    */
-  Token<ClientTokenIdentifier> getClientToken();
+  Token<ClientToAMTokenIdentifier> getClientToAMToken();
 
   /**
    * Diagnostics information for the application attempt.

+ 13 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/RMAppAttemptImpl.java

@@ -63,8 +63,8 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.security.AMRMTokenSelector;
-import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.ClientTokenSelector;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSelector;
 import org.apache.hadoop.yarn.server.resourcemanager.ApplicationMasterService;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
 import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEvent;
@@ -129,7 +129,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   private final WriteLock writeLock;
 
   private final ApplicationAttemptId applicationAttemptId;
-  private Token<ClientTokenIdentifier> clientToken;
+  private Token<ClientToAMTokenIdentifier> clientToAMToken;
   private final ApplicationSubmissionContext submissionContext;
   private Token<AMRMTokenIdentifier> amrmToken = null;
 
@@ -498,8 +498,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
   }
 
   @Override
-  public Token<ClientTokenIdentifier> getClientToken() {
-    return this.clientToken;
+  public Token<ClientToAMTokenIdentifier> getClientToAMToken() {
+    return this.clientToAMToken;
   }
 
   @Override
@@ -673,9 +673,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
     }
     if (UserGroupInformation.isSecurityEnabled()) {
 
-      ClientTokenSelector clientTokenSelector = new ClientTokenSelector();
-      this.clientToken =
-          clientTokenSelector.selectToken(new Text(),
+      ClientToAMTokenSelector clientToAMTokenSelector =
+          new ClientToAMTokenSelector();
+      this.clientToAMToken =
+          clientToAMTokenSelector.selectToken(new Text(),
             appAttemptTokens.getAllTokens());
 
       InetSocketAddress serviceAddr = conf.getSocketAddr(
@@ -720,9 +721,9 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
         appAttempt.rmContext.getClientToAMTokenSecretManager()
           .registerApplication(appAttempt.applicationAttemptId);
 
-        // create clientToken
-        appAttempt.clientToken =
-            new Token<ClientTokenIdentifier>(new ClientTokenIdentifier(
+        // create clientToAMToken
+        appAttempt.clientToAMToken =
+            new Token<ClientToAMTokenIdentifier>(new ClientToAMTokenIdentifier(
               appAttempt.applicationAttemptId),
               appAttempt.rmContext.getClientToAMTokenSecretManager());
 
@@ -1050,7 +1051,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       appAttempt.rmContext.getAMFinishingMonitor().unregister(
           appAttempt.getAppAttemptId());
 
-      // Unregister from the ClientTokenSecretManager
+      // Unregister from the ClientToAMTokenSecretManager
       if (UserGroupInformation.isSecurityEnabled()) {
         appAttempt.rmContext.getClientToAMTokenSecretManager()
           .unRegisterApplication(appAttempt.getAppAttemptId());

+ 4 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockAM.java

@@ -28,6 +28,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.FinishApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
@@ -49,7 +50,7 @@ public class MockAM {
   private final List<ResourceRequest> requests = new ArrayList<ResourceRequest>();
   private final List<ContainerId> releases = new ArrayList<ContainerId>();
 
-  MockAM(RMContext context, ApplicationMasterProtocol amRMProtocol,
+  public MockAM(RMContext context, ApplicationMasterProtocol amRMProtocol,
       ApplicationAttemptId attemptId) {
     this.context = context;
     this.amRMProtocol = amRMProtocol;
@@ -77,7 +78,7 @@ public class MockAM {
         finalState, attempt.getAppAttemptState());
   }
 
-  public void registerAppAttempt() throws Exception {
+  public RegisterApplicationMasterResponse registerAppAttempt() throws Exception {
     waitForState(RMAppAttemptState.LAUNCHED);
     responseId = 0;
     RegisterApplicationMasterRequest req = Records.newRecord(RegisterApplicationMasterRequest.class);
@@ -85,7 +86,7 @@ public class MockAM {
     req.setHost("");
     req.setRpcPort(1);
     req.setTrackingUrl("");
-    amRMProtocol.registerApplicationMaster(req);
+    return amRMProtocol.registerApplicationMaster(req);
   }
 
   public void addRequests(String[] hosts, int memory, int priority,

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java

@@ -96,7 +96,7 @@ public class MockRM extends ResourceManager {
     while (!finalState.equals(app.getState()) && timeoutSecs++ < 40) {
       System.out.println("App : " + appId + " State is : " + app.getState()
           + " Waiting for state : " + finalState);
-      Thread.sleep(1000);
+      Thread.sleep(2000);
     }
     System.out.println("App State is : " + app.getState());
     Assert.assertEquals("App state is not correct (timedout)", finalState,

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationACLs.java

@@ -338,7 +338,7 @@ public class TestApplicationACLs {
     Assert.assertEquals("Enemy should not see app rpc port!",
         -1, appReport.getRpcPort());
     Assert.assertEquals("Enemy should not see app client token!",
-        null, appReport.getClientToken());
+        null, appReport.getClientToAMToken());
     Assert.assertEquals("Enemy should not see app diagnostics!",
         UNAVAILABLE, appReport.getDiagnostics());
     Assert.assertEquals("Enemy should not see app tracking url!",

+ 8 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java

@@ -534,10 +534,11 @@ public class TestRMRestart {
     Assert.assertEquals(BuilderUtils.newContainerId(attemptId1, 1),
       attemptState.getMasterContainer().getId());
 
-    // the appToken and clientToken that are generated when RMAppAttempt is created,
+    // the appToken and clientToAMToken that are generated when RMAppAttempt
+    // is created,
     HashSet<Token<?>> tokenSet = new HashSet<Token<?>>();
     tokenSet.add(attempt1.getAMRMToken());
-    tokenSet.add(attempt1.getClientToken());
+    tokenSet.add(attempt1.getClientToAMToken());
 
     // assert application Token is saved
     HashSet<Token<?>> savedTokens = new HashSet<Token<?>>();
@@ -556,12 +557,13 @@ public class TestRMRestart {
     Assert.assertNotNull(loadedAttempt1);
     savedTokens.clear();
     savedTokens.add(loadedAttempt1.getAMRMToken());
-    savedTokens.add(loadedAttempt1.getClientToken());
+    savedTokens.add(loadedAttempt1.getClientToAMToken());
     Assert.assertEquals(tokenSet, savedTokens);
 
-    // assert clientToken is recovered back to api-versioned clientToken
-    Assert.assertEquals(attempt1.getClientToken(),
-      loadedAttempt1.getClientToken());
+    // assert clientToAMToken is recovered back to api-versioned
+    // clientToAMToken
+    Assert.assertEquals(attempt1.getClientToAMToken(),
+      loadedAttempt1.getClientToAMToken());
 
     // Not testing ApplicationTokenSecretManager has the password populated back,
     // that is needed in work-preserving restart

+ 0 - 35
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestSchedulerNegotiator.java

@@ -18,41 +18,6 @@
 
 package org.apache.hadoop.yarn.server.resourcemanager.applicationsmanager;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import junit.framework.Assert;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
-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.Priority;
-import org.apache.hadoop.yarn.api.records.QueueInfo;
-import org.apache.hadoop.yarn.api.records.QueueUserACLInfo;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.api.records.ResourceRequest;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
-import org.apache.hadoop.yarn.server.resourcemanager.RMContextImpl;
-import org.apache.hadoop.yarn.server.resourcemanager.amlauncher.AMLauncherEventType;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.RMState;
-import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStoreFactory;
-import org.apache.hadoop.yarn.server.resourcemanager.resource.Resources;
-import org.apache.hadoop.yarn.server.resourcemanager.rmapp.attempt.AMLivelinessMonitor;
-import org.apache.hadoop.yarn.server.resourcemanager.rmnode.RMNode;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.Allocation;
-import org.apache.hadoop.yarn.server.resourcemanager.scheduler.ResourceScheduler;
-import org.apache.hadoop.yarn.server.security.BaseContainerTokenSecretManager;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
 
 public class TestSchedulerNegotiator {
 //  private static RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);

+ 16 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/recovery/TestRMStateStore.java

@@ -55,7 +55,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.RMDelegationTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationAttemptState;
 import org.apache.hadoop.yarn.server.resourcemanager.recovery.RMStateStore.ApplicationState;
@@ -206,7 +206,7 @@ public class TestRMStateStore {
 
   ContainerId storeAttempt(RMStateStore store, ApplicationAttemptId attemptId,
       String containerIdStr, Token<AMRMTokenIdentifier> appToken,
-      Token<ClientTokenIdentifier> clientToken, TestDispatcher dispatcher)
+      Token<ClientToAMTokenIdentifier> clientToAMToken, TestDispatcher dispatcher)
       throws Exception {
 
     Container container = new ContainerPBImpl();
@@ -215,7 +215,7 @@ public class TestRMStateStore {
     when(mockAttempt.getAppAttemptId()).thenReturn(attemptId);
     when(mockAttempt.getMasterContainer()).thenReturn(container);
     when(mockAttempt.getAMRMToken()).thenReturn(appToken);
-    when(mockAttempt.getClientToken()).thenReturn(clientToken);
+    when(mockAttempt.getClientToAMToken()).thenReturn(clientToAMToken);
     dispatcher.attemptId = attemptId;
     dispatcher.storedException = null;
     store.storeApplicationAttempt(mockAttempt);
@@ -233,7 +233,7 @@ public class TestRMStateStore {
 
     AMRMTokenSecretManager appTokenMgr =
         new AMRMTokenSecretManager(conf);
-    ClientToAMTokenSecretManagerInRM clientTokenMgr =
+    ClientToAMTokenSecretManagerInRM clientToAMTokenMgr =
         new ClientToAMTokenSecretManagerInRM();
 
     ApplicationAttemptId attemptId1 = ConverterUtils
@@ -243,14 +243,14 @@ public class TestRMStateStore {
 
     // create application token1 for attempt1
     List<Token<?>> appAttemptToken1 =
-        generateTokens(attemptId1, appTokenMgr, clientTokenMgr, conf);
+        generateTokens(attemptId1, appTokenMgr, clientToAMTokenMgr, conf);
     HashSet<Token<?>> attemptTokenSet1 = new HashSet<Token<?>>();
     attemptTokenSet1.addAll(appAttemptToken1);
 
     ContainerId containerId1 = storeAttempt(store, attemptId1,
           "container_1352994193343_0001_01_000001",
           (Token<AMRMTokenIdentifier>) (appAttemptToken1.get(0)),
-          (Token<ClientTokenIdentifier>)(appAttemptToken1.get(1)),
+          (Token<ClientToAMTokenIdentifier>)(appAttemptToken1.get(1)),
           dispatcher);
 
     String appAttemptIdStr2 = "appattempt_1352994193343_0001_000002";
@@ -259,14 +259,14 @@ public class TestRMStateStore {
 
     // create application token2 for attempt2
     List<Token<?>> appAttemptToken2 =
-        generateTokens(attemptId2, appTokenMgr, clientTokenMgr, conf);
+        generateTokens(attemptId2, appTokenMgr, clientToAMTokenMgr, conf);
     HashSet<Token<?>> attemptTokenSet2 = new HashSet<Token<?>>();
     attemptTokenSet2.addAll(appAttemptToken2);
 
     ContainerId containerId2 = storeAttempt(store, attemptId2,
           "container_1352994193343_0001_02_000001",
           (Token<AMRMTokenIdentifier>) (appAttemptToken2.get(0)),
-          (Token<ClientTokenIdentifier>)(appAttemptToken2.get(1)),
+          (Token<ClientToAMTokenIdentifier>)(appAttemptToken2.get(1)),
           dispatcher);
 
     ApplicationAttemptId attemptIdRemoved = ConverterUtils
@@ -373,21 +373,22 @@ public class TestRMStateStore {
 
   private List<Token<?>> generateTokens(ApplicationAttemptId attemptId,
       AMRMTokenSecretManager appTokenMgr,
-      ClientToAMTokenSecretManagerInRM clientTokenMgr, Configuration conf) {
+      ClientToAMTokenSecretManagerInRM clientToAMTokenMgr, Configuration conf) {
     AMRMTokenIdentifier appTokenId =
         new AMRMTokenIdentifier(attemptId);
     Token<AMRMTokenIdentifier> appToken =
         new Token<AMRMTokenIdentifier>(appTokenId, appTokenMgr);
     appToken.setService(new Text("appToken service"));
 
-    ClientTokenIdentifier clientTokenId = new ClientTokenIdentifier(attemptId);
-    clientTokenMgr.registerApplication(attemptId);
-    Token<ClientTokenIdentifier> clientToken =
-        new Token<ClientTokenIdentifier>(clientTokenId, clientTokenMgr);
-    clientToken.setService(new Text("clientToken service"));
+    ClientToAMTokenIdentifier clientToAMTokenId =
+        new ClientToAMTokenIdentifier(attemptId);
+    clientToAMTokenMgr.registerApplication(attemptId);
+    Token<ClientToAMTokenIdentifier> clientToAMToken =
+        new Token<ClientToAMTokenIdentifier>(clientToAMTokenId, clientToAMTokenMgr);
+    clientToAMToken.setService(new Text("clientToAMToken service"));
     List<Token<?>> tokenPair = new ArrayList<Token<?>>();
     tokenPair.add(0, appToken);
-    tokenPair.add(1, clientToken);
+    tokenPair.add(1, clientToAMToken);
     return tokenPair;
   }
 }

+ 62 - 64
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientTokens.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java

@@ -19,17 +19,18 @@
 package org.apache.hadoop.yarn.server.resourcemanager.security;
 
 import static org.junit.Assert.fail;
+import static org.mockito.Mockito.mock;
 
 import java.io.IOException;
 import java.lang.annotation.Annotation;
 import java.net.InetSocketAddress;
+import java.security.PrivilegedAction;
 import java.security.PrivilegedExceptionAction;
 
 import javax.security.sasl.SaslException;
 
 import junit.framework.Assert;
 
-import org.apache.commons.codec.binary.Base64;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.ipc.RPC;
@@ -45,26 +46,21 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.security.token.TokenInfo;
 import org.apache.hadoop.security.token.TokenSelector;
 import org.apache.hadoop.service.AbstractService;
-import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenIdentifier;
 import org.apache.hadoop.yarn.security.client.ClientToAMTokenSecretManager;
-import org.apache.hadoop.yarn.security.client.ClientTokenIdentifier;
-import org.apache.hadoop.yarn.security.client.ClientTokenSelector;
+import org.apache.hadoop.yarn.security.client.ClientToAMTokenSelector;
 import org.apache.hadoop.yarn.server.resourcemanager.ClientRMService;
+import org.apache.hadoop.yarn.server.resourcemanager.MockAM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockNM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRM;
 import org.apache.hadoop.yarn.server.resourcemanager.MockRMWithCustomAMLauncher;
@@ -74,7 +70,7 @@ import org.apache.hadoop.yarn.util.ProtoUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Test;
 
-public class TestClientTokens {
+public class TestClientToAMTokens {
 
   private interface CustomProtocol {
     @SuppressWarnings("unused")
@@ -97,7 +93,7 @@ public class TestClientTokens {
         @Override
         public Class<? extends TokenSelector<? extends TokenIdentifier>>
             value() {
-          return ClientTokenSelector.class;
+          return ClientToAMTokenSelector.class;
         }
       };
     }
@@ -112,14 +108,15 @@ public class TestClientTokens {
       CustomProtocol {
 
     private final ApplicationAttemptId appAttemptId;
-    private final String secretKey;
+    private final byte[] secretKey;
     private InetSocketAddress address;
     private boolean pinged = false;
-
-    public CustomAM(ApplicationAttemptId appId, String secretKeyStr) {
+    private ClientToAMTokenSecretManager secretManager;
+    
+    public CustomAM(ApplicationAttemptId appId, byte[] secretKey) {
       super("CustomAM");
       this.appAttemptId = appId;
-      this.secretKey = secretKeyStr;
+      this.secretKey = secretKey;
     }
 
     @Override
@@ -131,9 +128,7 @@ public class TestClientTokens {
     protected void serviceStart() throws Exception {
       Configuration conf = getConfig();
 
-      ClientToAMTokenSecretManager secretManager = null;
-      byte[] bytes = Base64.decodeBase64(this.secretKey);
-      secretManager = new ClientToAMTokenSecretManager(this.appAttemptId, bytes);
+      secretManager = new ClientToAMTokenSecretManager(this.appAttemptId, secretKey);
       Server server;
       try {
         server =
@@ -147,44 +142,22 @@ public class TestClientTokens {
       this.address = NetUtils.getConnectAddress(server);
       super.serviceStart();
     }
-  }
-
-  private static class CustomNM implements ContainerManagementProtocol {
-
-    public String clientTokensSecret;
-
-    @Override
-    public StartContainerResponse startContainer(StartContainerRequest request)
-        throws YarnException {
-      this.clientTokensSecret =
-          request.getContainerLaunchContext().getEnvironment()
-            .get(ApplicationConstants.APPLICATION_CLIENT_SECRET_ENV_NAME);
-      return null;
-    }
-
-    @Override
-    public StopContainerResponse stopContainer(StopContainerRequest request)
-        throws YarnException {
-      return null;
-    }
-
-    @Override
-    public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request) throws YarnException {
-      return null;
+    
+    public ClientToAMTokenSecretManager getClientToAMTokenSecretManager() {
+      return this.secretManager;
     }
-
   }
 
   @Test
-  public void testClientTokens() throws Exception {
+  public void testClientToAMs() throws Exception {
 
     final Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
       "kerberos");
     UserGroupInformation.setConfiguration(conf);
 
-    CustomNM containerManager = new CustomNM();
+    ContainerManagementProtocol containerManager =
+        mock(ContainerManagementProtocol.class);
     final DrainDispatcher dispatcher = new DrainDispatcher();
 
     MockRM rm = new MockRMWithCustomAMLauncher(conf, containerManager) {
@@ -207,12 +180,15 @@ public class TestClientTokens {
 
     // Submit an app
     RMApp app = rm.submitApp(1024);
-    dispatcher.await();
 
     // Set up a node.
     MockNM nm1 = rm.registerNode("localhost:1234", 3072);
     nm1.nodeHeartbeat(true);
     dispatcher.await();
+    
+
+    nm1.nodeHeartbeat(true);
+    dispatcher.await();
 
     // Get the app-report.
     GetApplicationReportRequest request =
@@ -221,21 +197,43 @@ public class TestClientTokens {
     GetApplicationReportResponse reportResponse =
         rm.getClientRMService().getApplicationReport(request);
     ApplicationReport appReport = reportResponse.getApplicationReport();
-    org.apache.hadoop.yarn.api.records.Token clientToken = appReport.getClientToken();
-
-    // Wait till AM is 'launched'
-    int waitTime = 0;
-    while (containerManager.clientTokensSecret == null && waitTime++ < 20) {
-      Thread.sleep(1000);
-    }
-    Assert.assertNotNull(containerManager.clientTokensSecret);
-
+    org.apache.hadoop.yarn.api.records.Token clientToAMToken =
+        appReport.getClientToAMToken();
+
+    ApplicationAttemptId appAttempt = app.getCurrentAppAttempt().getAppAttemptId();
+    final MockAM mockAM =
+        new MockAM(rm.getRMContext(), rm.getApplicationMasterService(),
+            app.getCurrentAppAttempt().getAppAttemptId());
+    UserGroupInformation appUgi =
+        UserGroupInformation.createRemoteUser(appAttempt.toString());
+    RegisterApplicationMasterResponse response =
+        appUgi.doAs(new PrivilegedAction<RegisterApplicationMasterResponse>() {
+
+          @Override
+          public RegisterApplicationMasterResponse run() {
+            RegisterApplicationMasterResponse response = null;
+            try {
+              response = mockAM.registerAppAttempt();
+            } catch (Exception e) {
+              Assert.fail("Exception was not expected");
+            }
+            return response;
+          }
+        });
+     
+    // ClientToAMToken master key should have been received on register
+    // application master response.
+    Assert.assertNotNull(response.getClientToAMTokenMasterKey());
+    Assert
+        .assertTrue(response.getClientToAMTokenMasterKey().array().length > 0);
+    
     // Start the AM with the correct shared-secret.
     ApplicationAttemptId appAttemptId =
         app.getAppAttempts().keySet().iterator().next();
     Assert.assertNotNull(appAttemptId);
     final CustomAM am =
-        new CustomAM(appAttemptId, containerManager.clientTokensSecret);
+        new CustomAM(appAttemptId, response.getClientToAMTokenMasterKey()
+            .array());
     am.init(conf);
     am.start();
 
@@ -256,17 +254,17 @@ public class TestClientTokens {
 
     // Verify denial for a malicious user
     UserGroupInformation ugi = UserGroupInformation.createRemoteUser("me");
-    Token<ClientTokenIdentifier> token =
-        ProtoUtils.convertFromProtoFormat(clientToken, am.address);
+    Token<ClientToAMTokenIdentifier> token =
+        ProtoUtils.convertFromProtoFormat(clientToAMToken, am.address);
 
     // Malicious user, messes with appId
-    ClientTokenIdentifier maliciousID =
-        new ClientTokenIdentifier(BuilderUtils.newApplicationAttemptId(
+    ClientToAMTokenIdentifier maliciousID =
+        new ClientToAMTokenIdentifier(BuilderUtils.newApplicationAttemptId(
           BuilderUtils.newApplicationId(app.getApplicationId()
             .getClusterTimestamp(), 42), 43));
 
-    Token<ClientTokenIdentifier> maliciousToken =
-        new Token<ClientTokenIdentifier>(maliciousID.getBytes(),
+    Token<ClientToAMTokenIdentifier> maliciousToken =
+        new Token<ClientToAMTokenIdentifier>(maliciousID.getBytes(),
           token.getPassword(), token.getKind(),
           token.getService());
     ugi.addToken(maliciousToken);