浏览代码

git-svn-id: https://svn.apache.org/repos/asf/hadoop/common/branches/branch-2@1467130 13f79535-47bb-0310-9956-ffa450edef68

Vinod Kumar Vavilapalli 12 年之前
父节点
当前提交
a84c8bcbd4
共有 54 个文件被更改,包括 449 次插入416 次删除
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 10 12
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  3. 5 0
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
  4. 9 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerRemoteLaunchEvent.java
  5. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java
  6. 1 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
  7. 3 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/main/java/org/apache/hadoop/mapred/YARNRunner.java
  8. 4 0
      hadoop-yarn-project/CHANGES.txt
  9. 9 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
  10. 37 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerRequestPBImpl.java
  11. 8 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java
  12. 0 35
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
  13. 39 20
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java
  14. 3 76
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java
  15. 14 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_protos.proto
  16. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  17. 1 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-distributedshell/src/main/java/org/apache/hadoop/yarn/applications/distributedshell/ApplicationMaster.java
  18. 2 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 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java
  20. 7 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
  21. 9 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  22. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java
  23. 13 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  24. 3 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java
  25. 39 26
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java
  26. 7 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java
  27. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java
  28. 8 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
  29. 7 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
  30. 11 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
  31. 14 10
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  32. 24 18
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  33. 8 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java
  34. 14 9
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java
  35. 7 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java
  36. 8 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java
  37. 6 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java
  38. 11 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java
  39. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java
  40. 5 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/RMAppManager.java
  41. 20 21
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  42. 6 9
      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
  43. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java
  44. 4 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  45. 3 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockRM.java
  46. 9 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  47. 5 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java
  48. 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
  49. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
  50. 6 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  51. 3 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  52. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  53. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java
  54. 21 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

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

@@ -138,6 +138,9 @@ Release 2.0.5-beta - UNRELEASED
     MAPREDUCE-5136. TestJobImpl->testJobNoTasks fails with IBM JAVA (Amir
     MAPREDUCE-5136. TestJobImpl->testJobNoTasks fails with IBM JAVA (Amir
     Sanjar via jlowe)
     Sanjar via jlowe)
 
 
+    MAPREDUCE-5139. Update MR AM to use the modified startContainer API after
+    YARN-486. (Xuan Gong via vinodkv)
+
 Release 2.0.4-alpha - UNRELEASED
 Release 2.0.4-alpha - UNRELEASED
 
 
   INCOMPATIBLE CHANGES
   INCOMPATIBLE CHANGES

+ 10 - 12
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java

@@ -114,6 +114,7 @@ import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.api.records.ContainerToken;
@@ -767,8 +768,8 @@ public abstract class TaskAttemptImpl implements
     // The null fields are per-container and will be constructed for each
     // The null fields are per-container and will be constructed for each
     // container separately.
     // container separately.
     ContainerLaunchContext container = BuilderUtils
     ContainerLaunchContext container = BuilderUtils
-        .newContainerLaunchContext(null, conf
-            .get(MRJobConfig.USER_NAME), null, localResources,
+        .newContainerLaunchContext(conf
+            .get(MRJobConfig.USER_NAME), localResources,
             environment, null, serviceData, taskCredentialsBuffer,
             environment, null, serviceData, taskCredentialsBuffer,
             applicationACLs);
             applicationACLs);
 
 
@@ -777,10 +778,9 @@ public abstract class TaskAttemptImpl implements
 
 
   static ContainerLaunchContext createContainerLaunchContext(
   static ContainerLaunchContext createContainerLaunchContext(
       Map<ApplicationAccessType, String> applicationACLs,
       Map<ApplicationAccessType, String> applicationACLs,
-      ContainerId containerID, Configuration conf,
-      Token<JobTokenIdentifier> jobToken, Task remoteTask,
+      Configuration conf, Token<JobTokenIdentifier> jobToken, Task remoteTask,
       final org.apache.hadoop.mapred.JobID oldJobId,
       final org.apache.hadoop.mapred.JobID oldJobId,
-      Resource assignedCapability, WrappedJvmID jvmID,
+      WrappedJvmID jvmID,
       TaskAttemptListener taskAttemptListener,
       TaskAttemptListener taskAttemptListener,
       Credentials credentials) {
       Credentials credentials) {
 
 
@@ -813,7 +813,7 @@ public abstract class TaskAttemptImpl implements
 
 
     // Construct the actual Container
     // Construct the actual Container
     ContainerLaunchContext container = BuilderUtils.newContainerLaunchContext(
     ContainerLaunchContext container = BuilderUtils.newContainerLaunchContext(
-        containerID, commonContainerSpec.getUser(), assignedCapability,
+        commonContainerSpec.getUser(),
         commonContainerSpec.getLocalResources(), myEnv, commands,
         commonContainerSpec.getLocalResources(), myEnv, commands,
         myServiceData, commonContainerSpec.getContainerTokens().duplicate(),
         myServiceData, commonContainerSpec.getContainerTokens().duplicate(),
         applicationACLs);
         applicationACLs);
@@ -1511,15 +1511,13 @@ public abstract class TaskAttemptImpl implements
       //launch the container
       //launch the container
       //create the container object to be launched for a given Task attempt
       //create the container object to be launched for a given Task attempt
       ContainerLaunchContext launchContext = createContainerLaunchContext(
       ContainerLaunchContext launchContext = createContainerLaunchContext(
-          cEvent.getApplicationACLs(), taskAttempt.containerID,
-          taskAttempt.conf, taskAttempt.jobToken, taskAttempt.remoteTask,
-          taskAttempt.oldJobId, taskAttempt.assignedCapability,
-          taskAttempt.jvmID, taskAttempt.taskAttemptListener,
-          taskAttempt.credentials);
+          cEvent.getApplicationACLs(), taskAttempt.conf, taskAttempt.jobToken,
+          taskAttempt.remoteTask, taskAttempt.oldJobId, taskAttempt.jvmID,
+          taskAttempt.taskAttemptListener, taskAttempt.credentials);
       taskAttempt.eventHandler.handle(new ContainerRemoteLaunchEvent(
       taskAttempt.eventHandler.handle(new ContainerRemoteLaunchEvent(
           taskAttempt.attemptId, taskAttempt.containerID,
           taskAttempt.attemptId, taskAttempt.containerID,
           taskAttempt.containerMgrAddress, taskAttempt.containerToken,
           taskAttempt.containerMgrAddress, taskAttempt.containerToken,
-          launchContext, taskAttempt.remoteTask));
+          launchContext, taskAttempt.assignedCapability, taskAttempt.remoteTask));
 
 
       // send event to speculator that our container needs are satisfied
       // send event to speculator that our container needs are satisfied
       taskAttempt.eventHandler.handle
       taskAttempt.eventHandler.handle

+ 5 - 0
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java

@@ -59,6 +59,7 @@ import org.apache.hadoop.yarn.api.records.ContainerToken;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.service.AbstractService;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 import org.apache.hadoop.yarn.util.ProtoUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 
 
@@ -150,10 +151,14 @@ public class ContainerLauncherImpl extends AbstractService implements
         ContainerLaunchContext containerLaunchContext =
         ContainerLaunchContext containerLaunchContext =
           event.getContainer();
           event.getContainer();
 
 
+        org.apache.hadoop.yarn.api.records.Container container =
+            BuilderUtils.newContainer(containerID, null, null,
+                event.getResource(), null, containerToken);
         // Now launch the actual container
         // Now launch the actual container
         StartContainerRequest startRequest = Records
         StartContainerRequest startRequest = Records
           .newRecord(StartContainerRequest.class);
           .newRecord(StartContainerRequest.class);
         startRequest.setContainerLaunchContext(containerLaunchContext);
         startRequest.setContainerLaunchContext(containerLaunchContext);
+        startRequest.setContainer(container);
         StartContainerResponse response = proxy.startContainer(startRequest);
         StartContainerResponse response = proxy.startContainer(startRequest);
 
 
         ByteBuffer portInfo = response
         ByteBuffer portInfo = response

+ 9 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerRemoteLaunchEvent.java

@@ -23,26 +23,34 @@ import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
 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.ContainerToken;
 import org.apache.hadoop.yarn.api.records.ContainerToken;
+import org.apache.hadoop.yarn.api.records.Resource;
 
 
 public class ContainerRemoteLaunchEvent extends ContainerLauncherEvent {
 public class ContainerRemoteLaunchEvent extends ContainerLauncherEvent {
 
 
   private final ContainerLaunchContext container;
   private final ContainerLaunchContext container;
   private final Task task;
   private final Task task;
+  private final Resource resource;
 
 
   public ContainerRemoteLaunchEvent(TaskAttemptId taskAttemptID,
   public ContainerRemoteLaunchEvent(TaskAttemptId taskAttemptID,
       ContainerId containerID, String containerMgrAddress,
       ContainerId containerID, String containerMgrAddress,
       ContainerToken containerToken,
       ContainerToken containerToken,
-      ContainerLaunchContext containerLaunchContext, Task remoteTask) {
+      ContainerLaunchContext containerLaunchContext, Resource resource,
+      Task remoteTask) {
     super(taskAttemptID, containerID, containerMgrAddress, containerToken,
     super(taskAttemptID, containerID, containerMgrAddress, containerToken,
         ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH);
         ContainerLauncher.EventType.CONTAINER_REMOTE_LAUNCH);
     this.container = containerLaunchContext;
     this.container = containerLaunchContext;
     this.task = remoteTask;
     this.task = remoteTask;
+    this.resource = resource;
   }
   }
 
 
   public ContainerLaunchContext getContainer() {
   public ContainerLaunchContext getContainer() {
     return this.container;
     return this.container;
   }
   }
 
 
+  public Resource getResource() {
+    return this.resource;
+  }
+
   public Task getRemoteTask() {
   public Task getRemoteTask() {
     return this.task;
     return this.task;
   }
   }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttemptContainerRequest.java

@@ -113,9 +113,9 @@ public class TestTaskAttemptContainerRequest {
     ContainerId containerId = BuilderUtils.newContainerId(1, 1, 1, 1);
     ContainerId containerId = BuilderUtils.newContainerId(1, 1, 1, 1);
 
 
     ContainerLaunchContext launchCtx =
     ContainerLaunchContext launchCtx =
-        TaskAttemptImpl.createContainerLaunchContext(acls, containerId,
+        TaskAttemptImpl.createContainerLaunchContext(acls,
             jobConf, jobToken, taImpl.createRemoteTask(),
             jobConf, jobToken, taImpl.createRemoteTask(),
-            TypeConverter.fromYarn(jobId), mock(Resource.class),
+            TypeConverter.fromYarn(jobId),
             mock(WrappedJvmID.class), taListener,
             mock(WrappedJvmID.class), taListener,
             credentials);
             credentials);
 
 

+ 1 - 3
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java

@@ -60,7 +60,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 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;
-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.ContainerToken;
@@ -383,7 +382,6 @@ public class TestContainerLauncher {
     @Override
     @Override
     public StartContainerResponse startContainer(StartContainerRequest request)
     public StartContainerResponse startContainer(StartContainerRequest request)
         throws YarnRemoteException {
         throws YarnRemoteException {
-      ContainerLaunchContext container = request.getContainerLaunchContext();
       StartContainerResponse response = recordFactory
       StartContainerResponse response = recordFactory
           .newRecordInstance(StartContainerResponse.class);
           .newRecordInstance(StartContainerResponse.class);
       status = recordFactory.newRecordInstance(ContainerStatus.class);
       status = recordFactory.newRecordInstance(ContainerStatus.class);
@@ -395,7 +393,7 @@ public class TestContainerLauncher {
         throw new UndeclaredThrowableException(e);
         throw new UndeclaredThrowableException(e);
             }
             }
       status.setState(ContainerState.RUNNING);
       status.setState(ContainerState.RUNNING);
-      status.setContainerId(container.getContainerId());
+      status.setContainerId(request.getContainer().getId());
       status.setExitStatus(0);
       status.setExitStatus(0);
       return response;
       return response;
             }
             }

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

@@ -468,16 +468,14 @@ public class YARNRunner implements ClientProtocol {
 
 
     // Setup ContainerLaunchContext for AM container
     // Setup ContainerLaunchContext for AM container
     ContainerLaunchContext amContainer = BuilderUtils
     ContainerLaunchContext amContainer = BuilderUtils
-        .newContainerLaunchContext(null, UserGroupInformation
-            .getCurrentUser().getShortUserName(), capability, localResources,
+        .newContainerLaunchContext(UserGroupInformation
+            .getCurrentUser().getShortUserName(), localResources,
             environment, vargsFinal, null, securityTokens, acls);
             environment, vargsFinal, null, securityTokens, acls);
 
 
     // Set up the ApplicationSubmissionContext
     // Set up the ApplicationSubmissionContext
     ApplicationSubmissionContext appContext =
     ApplicationSubmissionContext appContext =
         recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
         recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     appContext.setApplicationId(applicationId);                // ApplicationId
     appContext.setApplicationId(applicationId);                // ApplicationId
-    appContext.setUser(                                        // User name
-        UserGroupInformation.getCurrentUser().getShortUserName());
     appContext.setQueue(                                       // Queue name
     appContext.setQueue(                                       // Queue name
         jobConf.get(JobContext.QUEUE_NAME,
         jobConf.get(JobContext.QUEUE_NAME,
         YarnConfiguration.DEFAULT_QUEUE_NAME));
         YarnConfiguration.DEFAULT_QUEUE_NAME));
@@ -490,7 +488,7 @@ public class YARNRunner implements ClientProtocol {
     appContext.setMaxAppAttempts(
     appContext.setMaxAppAttempts(
         conf.getInt(MRJobConfig.MR_AM_MAX_ATTEMPTS,
         conf.getInt(MRJobConfig.MR_AM_MAX_ATTEMPTS,
             MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS));
             MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS));
-
+    appContext.setResource(capability);
     return appContext;
     return appContext;
   }
   }
 
 

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

@@ -14,6 +14,10 @@ Release 2.0.5-beta - UNRELEASED
     YARN-536. Removed the unused objects ContainerStatus and ContainerStatus from
     YARN-536. Removed the unused objects ContainerStatus and ContainerStatus from
     Container which also don't belong to the container. (Xuan Gong via vinodkv)
     Container which also don't belong to the container. (Xuan Gong via vinodkv)
 
 
+    YARN-486. Changed NM's startContainer API to accept Container record given by
+    RM as a direct parameter instead of as part of the ContainerLaunchContext
+    record. (Xuan Gong via vinodkv)
+
   NEW FEATURES
   NEW FEATURES
 
 
   IMPROVEMENTS
   IMPROVEMENTS

+ 9 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java

@@ -21,6 +21,7 @@ package org.apache.hadoop.yarn.api.protocolrecords;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManager;
 import org.apache.hadoop.yarn.api.ContainerManager;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 
 
 /**
 /**
@@ -58,4 +59,12 @@ public interface StartContainerRequest {
   @Public
   @Public
   @Stable
   @Stable
   public abstract void setContainerLaunchContext(ContainerLaunchContext context);
   public abstract void setContainerLaunchContext(ContainerLaunchContext context);
+
+  @Public
+  @Stable
+  public Container getContainer();
+
+  @Public
+  @Stable
+  public void setContainer(Container container);
 }
 }

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

@@ -20,10 +20,13 @@ package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
 
 
 
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerLaunchContextPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProtoOrBuilder;
 
 
@@ -35,7 +38,8 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
   boolean viaProto = false;
   boolean viaProto = false;
   
   
   private ContainerLaunchContext containerLaunchContext = null;
   private ContainerLaunchContext containerLaunchContext = null;
-  
+
+  private Container container = null;
   
   
   public StartContainerRequestPBImpl() {
   public StartContainerRequestPBImpl() {
     builder = StartContainerRequestProto.newBuilder();
     builder = StartContainerRequestProto.newBuilder();
@@ -57,6 +61,9 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
     if (this.containerLaunchContext != null) {
     if (this.containerLaunchContext != null) {
       builder.setContainerLaunchContext(convertToProtoFormat(this.containerLaunchContext));
       builder.setContainerLaunchContext(convertToProtoFormat(this.containerLaunchContext));
     }
     }
+    if(this.container != null) {
+      builder.setContainer(convertToProtoFormat(this.container));
+    }
   }
   }
 
 
   private void mergeLocalToProto() {
   private void mergeLocalToProto() {
@@ -96,6 +103,28 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
     this.containerLaunchContext = containerLaunchContext;
     this.containerLaunchContext = containerLaunchContext;
   }
   }
 
 
+  @Override
+  public Container getContainer() {
+    StartContainerRequestProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.container != null) {
+      return this.container;
+    }
+    if (!p.hasContainer()) {
+      return null;
+    }
+    this.container = convertFromProtoFormat(p.getContainer());
+    return this.container;
+  }
+
+  @Override
+  public void setContainer(Container container) {
+    maybeInitBuilder();
+    if(container == null) {
+      builder.clearContainer();
+    }
+    this.container = container;
+  }
+
   private ContainerLaunchContextPBImpl convertFromProtoFormat(ContainerLaunchContextProto p) {
   private ContainerLaunchContextPBImpl convertFromProtoFormat(ContainerLaunchContextProto p) {
     return new ContainerLaunchContextPBImpl(p);
     return new ContainerLaunchContextPBImpl(p);
   }
   }
@@ -106,4 +135,11 @@ public class StartContainerRequestPBImpl extends ProtoBase<StartContainerRequest
 
 
 
 
 
 
+  private ContainerPBImpl convertFromProtoFormat(ContainerProto containerProto) {
+    return new ContainerPBImpl(containerProto);
+  }
+
+  private ContainerProto convertToProtoFormat(Container container) {
+    return ((ContainerPBImpl)container).getProto();
+  }
 }  
 }  

+ 8 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ApplicationSubmissionContext.java

@@ -112,22 +112,6 @@ public interface ApplicationSubmissionContext {
   @Public
   @Public
   @Stable
   @Stable
   public void setPriority(Priority priority);
   public void setPriority(Priority priority);
-  
-  /**
-   * Get the <em>user</em> submitting the application.
-   * @return <em>user</em> submitting the application
-   */
-  @Public
-  @Stable
-  public String getUser();
-  
-  /**
-   * Set the <em>user</em> submitting the application.
-   * @param user <em>user</em> submitting the application
-   */
-  @Public
-  @Stable
-  public void setUser(String user);
 
 
   /**
   /**
    * Get the <code>ContainerLaunchContext</code> to describe the 
    * Get the <code>ContainerLaunchContext</code> to describe the 
@@ -207,4 +191,12 @@ public interface ApplicationSubmissionContext {
   @Public
   @Public
   @Unstable
   @Unstable
   public void setMaxAppAttempts(int maxAppAttempts);
   public void setMaxAppAttempts(int maxAppAttempts);
+
+  @Public
+  @Stable
+  public Resource getResource();
+
+  @Public
+  @Stable
+  public void setResource(Resource resource);
 }
 }

+ 0 - 35
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java

@@ -51,22 +51,6 @@ import org.apache.hadoop.yarn.api.ContainerManager;
 @Public
 @Public
 @Stable
 @Stable
 public interface ContainerLaunchContext {
 public interface ContainerLaunchContext {
-  /**
-   * Get <code>ContainerId</code> of container to be launched.
-   * @return <code>ContainerId</code> of container to be launched
-   */
-  @Public
-  @Stable
-  ContainerId getContainerId();
-
-  /**
-   * Set <code>ContainerId</code> of container to be launched.
-   * @param containerId et <code>ContainerId</code> of container to be launched
-   */
-  @Public
-  @Stable
-  void setContainerId(ContainerId containerId);
-
   /**
   /**
    * Get the <em>user</em> to whom the container has been allocated.
    * Get the <em>user</em> to whom the container has been allocated.
    * @return the <em>user</em> to whom the container has been allocated
    * @return the <em>user</em> to whom the container has been allocated
@@ -83,25 +67,6 @@ public interface ContainerLaunchContext {
   @Stable
   @Stable
   void setUser(String user);
   void setUser(String user);
 
 
-  /**
-   * Get the <code>Resource</code> allocated to the container by the
-   * <code>ResourceManager</code>.
-   * @return <code>Resource</code> allocated to the container by the
-   *         <code>ResourceManager</code>
-   */
-  @Public
-  @Stable
-  Resource getResource();
-
-  /**
-   * Set the <code>Resource</code> allocated to the container by the
-   * <code>ResourceManager</code>.
-   * @param resource allocated resource
-   */
-  @Public
-  @Stable
-  void setResource(Resource resource);
-
   /**
   /**
    * Get security tokens (if security is enabled).
    * Get security tokens (if security is enabled).
    * @return security tokens (if security is enabled)
    * @return security tokens (if security is enabled)

+ 39 - 20
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ApplicationSubmissionContextPBImpl.java

@@ -23,11 +23,13 @@ import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationSubmissionContextProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.PriorityProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
     
     
 public class ApplicationSubmissionContextPBImpl 
 public class ApplicationSubmissionContextPBImpl 
 extends ProtoBase<ApplicationSubmissionContextProto> 
 extends ProtoBase<ApplicationSubmissionContextProto> 
@@ -40,7 +42,8 @@ implements ApplicationSubmissionContext {
   private ApplicationId applicationId = null;
   private ApplicationId applicationId = null;
   private Priority priority = null;
   private Priority priority = null;
   private ContainerLaunchContext amContainer = null;
   private ContainerLaunchContext amContainer = null;
-  
+  private Resource resource = null;
+
   public ApplicationSubmissionContextPBImpl() {
   public ApplicationSubmissionContextPBImpl() {
     builder = ApplicationSubmissionContextProto.newBuilder();
     builder = ApplicationSubmissionContextProto.newBuilder();
   }
   }
@@ -68,6 +71,11 @@ implements ApplicationSubmissionContext {
     if (this.amContainer != null) {
     if (this.amContainer != null) {
       builder.setAmContainerSpec(convertToProtoFormat(this.amContainer));
       builder.setAmContainerSpec(convertToProtoFormat(this.amContainer));
     }
     }
+    if (this.resource != null &&
+        !((ResourcePBImpl) this.resource).getProto().equals(
+            builder.getResource())) {
+      builder.setResource(convertToProtoFormat(this.resource));
+    }
   }
   }
 
 
   private void mergeLocalToProto() {
   private void mergeLocalToProto() {
@@ -165,25 +173,6 @@ implements ApplicationSubmissionContext {
     }
     }
     builder.setQueue((queue));
     builder.setQueue((queue));
   }
   }
-  
-  @Override
-  public String getUser() {
-    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
-    if (!p.hasUser()) {
-      return null;
-    }
-    return (p.getUser());
-  }
-
-  @Override
-  public void setUser(String user) {
-    maybeInitBuilder();
-    if (user == null) {
-      builder.clearUser();
-      return;
-    }
-    builder.setUser((user));
-  }
 
 
   @Override
   @Override
   public ContainerLaunchContext getAMContainerSpec() {
   public ContainerLaunchContext getAMContainerSpec() {
@@ -244,6 +233,28 @@ implements ApplicationSubmissionContext {
     builder.setMaxAppAttempts(maxAppAttempts);
     builder.setMaxAppAttempts(maxAppAttempts);
   }
   }
 
 
+  @Override
+  public Resource getResource() {
+    ApplicationSubmissionContextProtoOrBuilder p = viaProto ? proto : builder;
+    if (this.resource != null) {
+      return this.resource;
+    }
+    if (!p.hasResource()) {
+      return null;
+    }
+    this.resource = convertFromProtoFormat(p.getResource());
+    return this.resource;
+  }
+
+  @Override
+  public void setResource(Resource resource) {
+    maybeInitBuilder();
+    if (resource == null) {
+      builder.clearResource();
+    }
+    this.resource = resource;
+  }
+
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
   private PriorityPBImpl convertFromProtoFormat(PriorityProto p) {
     return new PriorityPBImpl(p);
     return new PriorityPBImpl(p);
   }
   }
@@ -268,4 +279,12 @@ implements ApplicationSubmissionContext {
   private ContainerLaunchContextProto convertToProtoFormat(ContainerLaunchContext t) {
   private ContainerLaunchContextProto convertToProtoFormat(ContainerLaunchContext t) {
     return ((ContainerLaunchContextPBImpl)t).getProto();
     return ((ContainerLaunchContextPBImpl)t).getProto();
   }
   }
+
+  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
+    return new ResourcePBImpl(p);
+  }
+
+  private ResourceProto convertToProtoFormat(Resource t) {
+    return ((ResourcePBImpl)t).getProto();
+  }
 }  
 }  

+ 3 - 76
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerLaunchContextPBImpl.java

@@ -26,17 +26,13 @@ import java.util.List;
 import java.util.Map;
 import java.util.Map;
 
 
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
-import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
 import org.apache.hadoop.yarn.api.records.ProtoBase;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationACLMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ApplicationACLMapProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerLaunchContextProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
-import org.apache.hadoop.yarn.proto.YarnProtos.ResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringBytesMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringBytesMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringLocalResourceMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringLocalResourceMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.StringStringMapProto;
@@ -50,8 +46,6 @@ implements ContainerLaunchContext {
   ContainerLaunchContextProto.Builder builder = null;
   ContainerLaunchContextProto.Builder builder = null;
   boolean viaProto = false;
   boolean viaProto = false;
   
   
-  private ContainerId containerId = null;
-  private Resource resource = null;
   private Map<String, LocalResource> localResources = null;
   private Map<String, LocalResource> localResources = null;
   private ByteBuffer containerTokens = null;
   private ByteBuffer containerTokens = null;
   private Map<String, ByteBuffer> serviceData = null;
   private Map<String, ByteBuffer> serviceData = null;
@@ -76,16 +70,6 @@ implements ContainerLaunchContext {
   }
   }
   
   
   private void mergeLocalToBuilder() {
   private void mergeLocalToBuilder() {
-    if (this.containerId != null && 
-        !((ContainerIdPBImpl)containerId).getProto().equals(
-            builder.getContainerId())) {
-      builder.setContainerId(convertToProtoFormat(this.containerId));
-    }
-    if (this.resource != null && 
-        !((ResourcePBImpl)this.resource).getProto().equals(
-            builder.getResource())) {
-      builder.setResource(convertToProtoFormat(this.resource));
-    }
     if (this.localResources != null) {
     if (this.localResources != null) {
       addLocalResourcesToProto();
       addLocalResourcesToProto();
     }
     }
@@ -120,28 +104,6 @@ implements ContainerLaunchContext {
     }
     }
     viaProto = false;
     viaProto = false;
   }
   }
-    
-  
-  @Override
-  public Resource getResource() {
-    ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.resource != null) {
-      return this.resource;
-    }
-    if (!p.hasResource()) {
-      return null;
-    }
-    this.resource = convertFromProtoFormat(p.getResource());
-    return this.resource;
-  }
-
-  @Override
-  public void setResource(Resource resource) {
-    maybeInitBuilder();
-    if (resource == null) 
-      builder.clearResource();
-    this.resource = resource;
-  }
   
   
   @Override
   @Override
   public List<String> getCommands() {
   public List<String> getCommands() {
@@ -197,26 +159,6 @@ implements ContainerLaunchContext {
     }
     }
     builder.setUser((user));
     builder.setUser((user));
   }
   }
-  @Override
-  public ContainerId getContainerId() {
-    ContainerLaunchContextProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.containerId != null) {
-      return this.containerId;
-    }
-    if (!p.hasContainerId()) {
-      return null;
-    }
-    this.containerId = convertFromProtoFormat(p.getContainerId());
-    return this.containerId;
-  }
-
-  @Override
-  public void setContainerId(ContainerId containerId) {
-    maybeInitBuilder();
-    if (containerId == null) 
-      builder.clearContainerId();
-    this.containerId = containerId;
-  }
   
   
   @Override
   @Override
   public Map<String, LocalResource> getLocalResources() {
   public Map<String, LocalResource> getLocalResources() {
@@ -299,11 +241,12 @@ implements ContainerLaunchContext {
   @Override
   @Override
   public void setContainerTokens(ByteBuffer containerTokens) {
   public void setContainerTokens(ByteBuffer containerTokens) {
     maybeInitBuilder();
     maybeInitBuilder();
-    if (containerTokens == null)
+    if (containerTokens == null) {
       builder.clearContainerTokens();
       builder.clearContainerTokens();
+    }
     this.containerTokens = containerTokens;
     this.containerTokens = containerTokens;
   }
   }
-  
+
   @Override
   @Override
   public Map<String, ByteBuffer> getServiceData() {
   public Map<String, ByteBuffer> getServiceData() {
     initServiceData();
     initServiceData();
@@ -500,22 +443,6 @@ implements ContainerLaunchContext {
     this.applicationACLS.putAll(appACLs);
     this.applicationACLS.putAll(appACLs);
   }
   }
 
 
-  private ResourcePBImpl convertFromProtoFormat(ResourceProto p) {
-    return new ResourcePBImpl(p);
-  }
-
-  private ResourceProto convertToProtoFormat(Resource t) {
-    return ((ResourcePBImpl)t).getProto();
-  }
-
-  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
-    return new ContainerIdPBImpl(p);
-  }
-
-  private ContainerIdProto convertToProtoFormat(ContainerId t) {
-    return ((ContainerIdPBImpl)t).getProto();
-  }
-
   private LocalResourcePBImpl convertFromProtoFormat(LocalResourceProto p) {
   private LocalResourcePBImpl convertFromProtoFormat(LocalResourceProto p) {
     return new LocalResourcePBImpl(p);
     return new LocalResourcePBImpl(p);
   }
   }

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

@@ -211,13 +211,13 @@ message ResourceRequestProto {
 message ApplicationSubmissionContextProto {
 message ApplicationSubmissionContextProto {
   optional ApplicationIdProto application_id = 1;
   optional ApplicationIdProto application_id = 1;
   optional string application_name = 2 [default = "N/A"];
   optional string application_name = 2 [default = "N/A"];
-  optional string user = 3;
-  optional string queue = 4 [default = "default"];
-  optional PriorityProto priority = 5;
-  optional ContainerLaunchContextProto am_container_spec = 6;
-  optional bool cancel_tokens_when_complete = 7 [default = true];
-  optional bool unmanaged_am = 8 [default = false];
-  optional int32 maxAppAttempts = 9 [default = 0];
+  optional string queue = 3 [default = "default"];
+  optional PriorityProto priority = 4;
+  optional ContainerLaunchContextProto am_container_spec = 5;
+  optional bool cancel_tokens_when_complete = 6 [default = true];
+  optional bool unmanaged_am = 7 [default = false];
+  optional int32 maxAppAttempts = 8 [default = 0];
+  optional ResourceProto resource = 9;
 }
 }
 
 
 enum ApplicationAccessTypeProto {
 enum ApplicationAccessTypeProto {
@@ -264,15 +264,13 @@ message QueueUserACLInfoProto {
 ////////////////////////////////////////////////////////////////////////
 ////////////////////////////////////////////////////////////////////////
 
 
 message ContainerLaunchContextProto {
 message ContainerLaunchContextProto {
-  optional ContainerIdProto container_id = 1;
-  optional string user = 2;
-  optional ResourceProto resource = 3;
-  repeated StringLocalResourceMapProto localResources = 4;
-  optional bytes container_tokens = 5;
-  repeated StringBytesMapProto service_data = 6;
-  repeated StringStringMapProto environment = 7;
-  repeated string command = 8;
-  repeated ApplicationACLMapProto application_ACLs = 9;
+  optional string user = 1;
+  repeated StringLocalResourceMapProto localResources = 2;
+  optional bytes container_tokens = 3;
+  repeated StringBytesMapProto service_data = 4;
+  repeated StringStringMapProto environment = 5;
+  repeated string command = 6;
+  repeated ApplicationACLMapProto application_ACLs = 7;
 }
 }
 
 
 message ContainerStatusProto {
 message ContainerStatusProto {

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

@@ -151,6 +151,7 @@ message GetQueueUserAclsInfoResponseProto {
 
 
 message StartContainerRequestProto {
 message StartContainerRequestProto {
   optional ContainerLaunchContextProto container_launch_context = 1;
   optional ContainerLaunchContextProto container_launch_context = 1;
+  optional ContainerProto container = 2;
 }
 }
 
 
 message StartContainerResponseProto {
 message StartContainerResponseProto {

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

@@ -679,9 +679,6 @@ public class ApplicationMaster {
       ContainerLaunchContext ctx = Records
       ContainerLaunchContext ctx = Records
           .newRecord(ContainerLaunchContext.class);
           .newRecord(ContainerLaunchContext.class);
 
 
-      ctx.setContainerId(container.getId());
-      ctx.setResource(container.getResource());
-
       String jobUserName = System.getenv(ApplicationConstants.Environment.USER
       String jobUserName = System.getenv(ApplicationConstants.Environment.USER
           .name());
           .name());
       ctx.setUser(jobUserName);
       ctx.setUser(jobUserName);
@@ -752,6 +749,7 @@ public class ApplicationMaster {
       StartContainerRequest startReq = Records
       StartContainerRequest startReq = Records
           .newRecord(StartContainerRequest.class);
           .newRecord(StartContainerRequest.class);
       startReq.setContainerLaunchContext(ctx);
       startReq.setContainerLaunchContext(ctx);
+      startReq.setContainer(container);
       try {
       try {
         cm.startContainer(startReq);
         cm.startContainer(startReq);
       } catch (YarnRemoteException e) {
       } catch (YarnRemoteException e) {

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

@@ -545,7 +545,7 @@ public class Client extends YarnClientImpl {
     // For now, only memory is supported so we set memory requirements
     // For now, only memory is supported so we set memory requirements
     Resource capability = Records.newRecord(Resource.class);
     Resource capability = Records.newRecord(Resource.class);
     capability.setMemory(amMemory);
     capability.setMemory(amMemory);
-    amContainer.setResource(capability);
+    appContext.setResource(capability);
 
 
     // Service data is a binary blob that can be passed to the application
     // Service data is a binary blob that can be passed to the application
     // Not needed in this scenario
     // Not needed in this scenario
@@ -570,6 +570,7 @@ public class Client extends YarnClientImpl {
     // Ignore the response as either a valid response object is returned on success 
     // Ignore the response as either a valid response object is returned on success 
     // or an exception thrown to denote some form of a failure
     // or an exception thrown to denote some form of a failure
     LOG.info("Submitting application to ASM");
     LOG.info("Submitting application to ASM");
+
     super.submitApplication(appContext);
     super.submitApplication(appContext);
 
 
     // TODO
     // TODO

+ 2 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/util/BuilderUtils.java

@@ -284,16 +284,13 @@ public class BuilderUtils {
   }
   }
 
 
   public static ContainerLaunchContext newContainerLaunchContext(
   public static ContainerLaunchContext newContainerLaunchContext(
-      ContainerId containerID, String user, Resource assignedCapability,
-      Map<String, LocalResource> localResources,
+      String user, Map<String, LocalResource> localResources,
       Map<String, String> environment, List<String> commands,
       Map<String, String> environment, List<String> commands,
-      Map<String, ByteBuffer> serviceData, ByteBuffer containerTokens,
+      Map<String, ByteBuffer> serviceData,  ByteBuffer containerTokens,
       Map<ApplicationAccessType, String> acls) {
       Map<ApplicationAccessType, String> acls) {
     ContainerLaunchContext container = recordFactory
     ContainerLaunchContext container = recordFactory
         .newRecordInstance(ContainerLaunchContext.class);
         .newRecordInstance(ContainerLaunchContext.class);
-    container.setContainerId(containerID);
     container.setUser(user);
     container.setUser(user);
-    container.setResource(assignedCapability);
     container.setLocalResources(localResources);
     container.setLocalResources(localResources);
     container.setEnvironment(environment);
     container.setEnvironment(environment);
     container.setCommands(commands);
     container.setCommands(commands);

+ 7 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java

@@ -38,6 +38,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 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.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -50,6 +51,7 @@ import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvider;
 import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvider;
 import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
 import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 import org.junit.Test;
 
 
 /*
 /*
@@ -101,13 +103,14 @@ public class TestContainerLaunchRPC {
       applicationAttemptId.setAttemptId(0);
       applicationAttemptId.setAttemptId(0);
       containerId.setApplicationAttemptId(applicationAttemptId);
       containerId.setApplicationAttemptId(applicationAttemptId);
       containerId.setId(100);
       containerId.setId(100);
-      containerLaunchContext.setContainerId(containerId);
-      containerLaunchContext.setResource(recordFactory
-          .newRecordInstance(Resource.class));
+      Container container =
+          BuilderUtils.newContainer(containerId, null, null, recordFactory
+              .newRecordInstance(Resource.class), null, null);
 
 
       StartContainerRequest scRequest = recordFactory
       StartContainerRequest scRequest = recordFactory
           .newRecordInstance(StartContainerRequest.class);
           .newRecordInstance(StartContainerRequest.class);
       scRequest.setContainerLaunchContext(containerLaunchContext);
       scRequest.setContainerLaunchContext(containerLaunchContext);
+      scRequest.setContainer(container);
       try {
       try {
         proxy.startContainer(scRequest);
         proxy.startContainer(scRequest);
       } catch (Exception e) {
       } catch (Exception e) {
@@ -141,7 +144,6 @@ public class TestContainerLaunchRPC {
     @Override
     @Override
     public StartContainerResponse startContainer(StartContainerRequest request)
     public StartContainerResponse startContainer(StartContainerRequest request)
         throws YarnRemoteException {
         throws YarnRemoteException {
-      ContainerLaunchContext container = request.getContainerLaunchContext();
       StartContainerResponse response = recordFactory
       StartContainerResponse response = recordFactory
           .newRecordInstance(StartContainerResponse.class);
           .newRecordInstance(StartContainerResponse.class);
       status = recordFactory.newRecordInstance(ContainerStatus.class);
       status = recordFactory.newRecordInstance(ContainerStatus.class);
@@ -153,7 +155,7 @@ public class TestContainerLaunchRPC {
         throw new UndeclaredThrowableException(e);
         throw new UndeclaredThrowableException(e);
       }
       }
       status.setState(ContainerState.RUNNING);
       status.setState(ContainerState.RUNNING);
-      status.setContainerId(container.getContainerId());
+      status.setContainerId(request.getContainer().getId());
       status.setExitStatus(0);
       status.setExitStatus(0);
       return response;
       return response;
     }
     }

+ 9 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 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.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -52,6 +53,7 @@ import org.apache.hadoop.yarn.factory.providers.YarnRemoteExceptionFactoryProvid
 import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
 import org.apache.hadoop.yarn.ipc.HadoopYarnProtoRPC;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.util.Records;
 import org.junit.Test;
 import org.junit.Test;
 
 
@@ -124,20 +126,21 @@ public class TestRPC {
     applicationAttemptId.setAttemptId(0);
     applicationAttemptId.setAttemptId(0);
     containerId.setApplicationAttemptId(applicationAttemptId);
     containerId.setApplicationAttemptId(applicationAttemptId);
     containerId.setId(100);
     containerId.setId(100);
-    containerLaunchContext.setContainerId(containerId);
-    containerLaunchContext.setResource(
-        recordFactory.newRecordInstance(Resource.class));
+    Container mockContainer =
+        BuilderUtils.newContainer(containerId, null, null, recordFactory
+            .newRecordInstance(Resource.class), null, null);
 //    containerLaunchContext.env = new HashMap<CharSequence, CharSequence>();
 //    containerLaunchContext.env = new HashMap<CharSequence, CharSequence>();
 //    containerLaunchContext.command = new ArrayList<CharSequence>();
 //    containerLaunchContext.command = new ArrayList<CharSequence>();
     
     
     StartContainerRequest scRequest = 
     StartContainerRequest scRequest = 
         recordFactory.newRecordInstance(StartContainerRequest.class);
         recordFactory.newRecordInstance(StartContainerRequest.class);
     scRequest.setContainerLaunchContext(containerLaunchContext);
     scRequest.setContainerLaunchContext(containerLaunchContext);
+    scRequest.setContainer(mockContainer);
     proxy.startContainer(scRequest);
     proxy.startContainer(scRequest);
     
     
     GetContainerStatusRequest gcsRequest = 
     GetContainerStatusRequest gcsRequest = 
         recordFactory.newRecordInstance(GetContainerStatusRequest.class);
         recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-    gcsRequest.setContainerId(containerLaunchContext.getContainerId());
+    gcsRequest.setContainerId(mockContainer.getId());
     GetContainerStatusResponse response =  proxy.getContainerStatus(gcsRequest);
     GetContainerStatusResponse response =  proxy.getContainerStatus(gcsRequest);
     ContainerStatus status = response.getStatus();
     ContainerStatus status = response.getStatus();
     
     
@@ -145,7 +148,7 @@ public class TestRPC {
     boolean exception = false;
     boolean exception = false;
     try {
     try {
       StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
       StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
-      stopRequest.setContainerId(containerLaunchContext.getContainerId());
+      stopRequest.setContainerId(mockContainer.getId());
       proxy.stopContainer(stopRequest);
       proxy.stopContainer(stopRequest);
     } catch (YarnRemoteException e) {
     } catch (YarnRemoteException e) {
       exception = true;
       exception = true;
@@ -179,12 +182,11 @@ public class TestRPC {
     @Override
     @Override
     public StartContainerResponse startContainer(StartContainerRequest request) 
     public StartContainerResponse startContainer(StartContainerRequest request) 
         throws YarnRemoteException {
         throws YarnRemoteException {
-      ContainerLaunchContext container = request.getContainerLaunchContext();
       StartContainerResponse response = 
       StartContainerResponse response = 
           recordFactory.newRecordInstance(StartContainerResponse.class);
           recordFactory.newRecordInstance(StartContainerResponse.class);
       status = recordFactory.newRecordInstance(ContainerStatus.class);
       status = recordFactory.newRecordInstance(ContainerStatus.class);
       status.setState(ContainerState.RUNNING);
       status.setState(ContainerState.RUNNING);
-      status.setContainerId(container.getContainerId());
+      status.setContainerId(request.getContainer().getId());
       status.setExitStatus(0);
       status.setExitStatus(0);
       return response;
       return response;
     }
     }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/LinuxContainerExecutor.java

@@ -221,7 +221,7 @@ public class LinuxContainerExecutor extends ContainerExecutor {
     String containerIdStr = ConverterUtils.toString(containerId);
     String containerIdStr = ConverterUtils.toString(containerId);
     
     
     resourcesHandler.preExecute(containerId,
     resourcesHandler.preExecute(containerId,
-            container.getLaunchContext().getResource());
+            container.getResource());
     String resourcesOptions = resourcesHandler.getResourcesOption(
     String resourcesOptions = resourcesHandler.getResourcesOption(
             containerId);
             containerId);
 
 

+ 13 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java

@@ -308,7 +308,9 @@ public class ContainerManagerImpl extends CompositeService implements
    * @throws YarnRemoteException
    * @throws YarnRemoteException
    */
    */
   private void authorizeRequest(String containerIDStr,
   private void authorizeRequest(String containerIDStr,
-      ContainerLaunchContext launchContext, UserGroupInformation remoteUgi)
+      ContainerLaunchContext launchContext,
+      org.apache.hadoop.yarn.api.records.Container container,
+      UserGroupInformation remoteUgi)
       throws YarnRemoteException {
       throws YarnRemoteException {
 
 
     if (!UserGroupInformation.isSecurityEnabled()) {
     if (!UserGroupInformation.isSecurityEnabled()) {
@@ -369,10 +371,10 @@ public class ContainerManagerImpl extends CompositeService implements
         }
         }
         
         
         Resource resource = tokenId.getResource();
         Resource resource = tokenId.getResource();
-        if (!resource.equals(launchContext.getResource())) {
+        if (!resource.equals(container.getResource())) {
           unauthorized = true;
           unauthorized = true;
           messageBuilder.append("\nExpected resource " + resource
           messageBuilder.append("\nExpected resource " + resource
-              + " but found " + launchContext.getResource());
+              + " but found " + container.getResource());
         }
         }
       }
       }
     }
     }
@@ -392,12 +394,13 @@ public class ContainerManagerImpl extends CompositeService implements
   public StartContainerResponse startContainer(StartContainerRequest request)
   public StartContainerResponse startContainer(StartContainerRequest request)
       throws YarnRemoteException {
       throws YarnRemoteException {
     ContainerLaunchContext launchContext = request.getContainerLaunchContext();
     ContainerLaunchContext launchContext = request.getContainerLaunchContext();
-
-    ContainerId containerID = launchContext.getContainerId();
+    org.apache.hadoop.yarn.api.records.Container lauchContainer =
+        request.getContainer();
+    ContainerId containerID = lauchContainer.getId();
     String containerIDStr = containerID.toString();
     String containerIDStr = containerID.toString();
 
 
     UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
     UserGroupInformation remoteUgi = getRemoteUgi(containerIDStr);
-    authorizeRequest(containerIDStr, launchContext, remoteUgi);
+    authorizeRequest(containerIDStr, launchContext, lauchContainer, remoteUgi);
 
 
     LOG.info("Start request for " + containerIDStr + " by user "
     LOG.info("Start request for " + containerIDStr + " by user "
         + launchContext.getUser());
         + launchContext.getUser());
@@ -424,7 +427,7 @@ public class ContainerManagerImpl extends CompositeService implements
     // //////////// End of parsing credentials
     // //////////// End of parsing credentials
 
 
     Container container = new ContainerImpl(getConfig(), this.dispatcher,
     Container container = new ContainerImpl(getConfig(), this.dispatcher,
-        launchContext, credentials, metrics);
+        launchContext, lauchContainer, credentials, metrics);
     ApplicationId applicationID = 
     ApplicationId applicationID = 
         containerID.getApplicationAttemptId().getApplicationId();
         containerID.getApplicationAttemptId().getApplicationId();
     if (context.getContainers().putIfAbsent(containerID, container) != null) {
     if (context.getContainers().putIfAbsent(containerID, container) != null) {
@@ -469,7 +472,7 @@ public class ContainerManagerImpl extends CompositeService implements
     // TODO launchedContainer misplaced -> doesn't necessarily mean a container
     // TODO launchedContainer misplaced -> doesn't necessarily mean a container
     // launch. A finished Application will not launch containers.
     // launch. A finished Application will not launch containers.
     metrics.launchedContainer();
     metrics.launchedContainer();
-    metrics.allocateContainer(launchContext.getResource());
+    metrics.allocateContainer(lauchContainer.getResource());
     return response;
     return response;
   }
   }
 
 
@@ -487,7 +490,7 @@ public class ContainerManagerImpl extends CompositeService implements
     // 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, remoteUgi);
+    authorizeRequest(containerIDStr, null, null, remoteUgi);
 
 
     StopContainerResponse response =
     StopContainerResponse response =
         recordFactory.newRecordInstance(StopContainerResponse.class);
         recordFactory.newRecordInstance(StopContainerResponse.class);
@@ -529,7 +532,7 @@ public class ContainerManagerImpl extends CompositeService implements
     // 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, remoteUgi);
+    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);

+ 3 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/Container.java

@@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.Credentials;
 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.Resource;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
 
 
 public interface Container extends EventHandler<ContainerEvent> {
 public interface Container extends EventHandler<ContainerEvent> {
@@ -44,4 +45,6 @@ public interface Container extends EventHandler<ContainerEvent> {
   ContainerStatus cloneAndGetContainerStatus();
   ContainerStatus cloneAndGetContainerStatus();
 
 
   String toString();
   String toString();
+
+  Resource getResource();
 }
 }

+ 39 - 26
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/ContainerImpl.java

@@ -41,6 +41,7 @@ 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.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+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.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -76,6 +77,7 @@ public class ContainerImpl implements Container {
   private final Credentials credentials;
   private final Credentials credentials;
   private final NodeManagerMetrics metrics;
   private final NodeManagerMetrics metrics;
   private final ContainerLaunchContext launchContext;
   private final ContainerLaunchContext launchContext;
+  private final org.apache.hadoop.yarn.api.records.Container container;
   private int exitCode = YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS;
   private int exitCode = YarnConfiguration.INVALID_CONTAINER_EXIT_STATUS;
   private final StringBuilder diagnostics;
   private final StringBuilder diagnostics;
 
 
@@ -96,12 +98,13 @@ public class ContainerImpl implements Container {
     new ArrayList<LocalResourceRequest>();
     new ArrayList<LocalResourceRequest>();
 
 
   public ContainerImpl(Configuration conf,
   public ContainerImpl(Configuration conf,
-      Dispatcher dispatcher,
-      ContainerLaunchContext launchContext, Credentials creds,
-      NodeManagerMetrics metrics) {
+      Dispatcher dispatcher, ContainerLaunchContext launchContext,
+      org.apache.hadoop.yarn.api.records.Container container,
+      Credentials creds, NodeManagerMetrics metrics) {
     this.daemonConf = conf;
     this.daemonConf = conf;
     this.dispatcher = dispatcher;
     this.dispatcher = dispatcher;
     this.launchContext = launchContext;
     this.launchContext = launchContext;
+    this.container = container;
     this.diagnostics = new StringBuilder();
     this.diagnostics = new StringBuilder();
     this.credentials = creds;
     this.credentials = creds;
     this.metrics = metrics;
     this.metrics = metrics;
@@ -312,7 +315,7 @@ public class ContainerImpl implements Container {
   public ContainerId getContainerID() {
   public ContainerId getContainerID() {
     this.readLock.lock();
     this.readLock.lock();
     try {
     try {
-      return this.launchContext.getContainerId();
+      return this.container.getId();
     } finally {
     } finally {
       this.readLock.unlock();
       this.readLock.unlock();
     }
     }
@@ -373,50 +376,61 @@ public class ContainerImpl implements Container {
   public ContainerStatus cloneAndGetContainerStatus() {
   public ContainerStatus cloneAndGetContainerStatus() {
     this.readLock.lock();
     this.readLock.lock();
     try {
     try {
-      return BuilderUtils.newContainerStatus(this.getContainerID(),
+      return BuilderUtils.newContainerStatus(this.container.getId(),
         getCurrentState(), diagnostics.toString(), exitCode);
         getCurrentState(), diagnostics.toString(), exitCode);
     } finally {
     } finally {
       this.readLock.unlock();
       this.readLock.unlock();
     }
     }
   }
   }
 
 
+  @Override
+  public Resource getResource() {
+    this.readLock.lock();
+    try {
+      return this.container.getResource();
+    } finally {
+      this.readLock.unlock();
+    }
+  }
+
   @SuppressWarnings({"fallthrough", "unchecked"})
   @SuppressWarnings({"fallthrough", "unchecked"})
   private void finished() {
   private void finished() {
+    ContainerId containerID = this.container.getId();
+    String user = this.launchContext.getUser();
     switch (getContainerState()) {
     switch (getContainerState()) {
       case EXITED_WITH_SUCCESS:
       case EXITED_WITH_SUCCESS:
         metrics.endRunningContainer();
         metrics.endRunningContainer();
         metrics.completedContainer();
         metrics.completedContainer();
-        NMAuditLogger.logSuccess(getUser(),
+        NMAuditLogger.logSuccess(user,
             AuditConstants.FINISH_SUCCESS_CONTAINER, "ContainerImpl",
             AuditConstants.FINISH_SUCCESS_CONTAINER, "ContainerImpl",
-            getContainerID().getApplicationAttemptId().getApplicationId(), 
-            getContainerID());
+            containerID.getApplicationAttemptId()
+                .getApplicationId(), containerID);
         break;
         break;
       case EXITED_WITH_FAILURE:
       case EXITED_WITH_FAILURE:
         metrics.endRunningContainer();
         metrics.endRunningContainer();
         // fall through
         // fall through
       case LOCALIZATION_FAILED:
       case LOCALIZATION_FAILED:
         metrics.failedContainer();
         metrics.failedContainer();
-        NMAuditLogger.logFailure(getUser(),
+        NMAuditLogger.logFailure(user,
             AuditConstants.FINISH_FAILED_CONTAINER, "ContainerImpl",
             AuditConstants.FINISH_FAILED_CONTAINER, "ContainerImpl",
             "Container failed with state: " + getContainerState(),
             "Container failed with state: " + getContainerState(),
-            getContainerID().getApplicationAttemptId().getApplicationId(), 
-            getContainerID());
+            containerID.getApplicationAttemptId()
+                .getApplicationId(), containerID);
         break;
         break;
       case CONTAINER_CLEANEDUP_AFTER_KILL:
       case CONTAINER_CLEANEDUP_AFTER_KILL:
         metrics.endRunningContainer();
         metrics.endRunningContainer();
         // fall through
         // fall through
       case NEW:
       case NEW:
         metrics.killedContainer();
         metrics.killedContainer();
-        NMAuditLogger.logSuccess(getUser(),
+        NMAuditLogger.logSuccess(user,
             AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
             AuditConstants.FINISH_KILLED_CONTAINER, "ContainerImpl",
-            getContainerID().getApplicationAttemptId().getApplicationId(), 
-            getContainerID());
+            containerID.getApplicationAttemptId().getApplicationId(),
+            containerID);
     }
     }
 
 
-    metrics.releaseContainer(getLaunchContext().getResource());
+    metrics.releaseContainer(this.container.getResource());
 
 
     // Inform the application
     // Inform the application
-    ContainerId containerID = getContainerID();
     @SuppressWarnings("rawtypes")
     @SuppressWarnings("rawtypes")
     EventHandler eventHandler = dispatcher.getEventHandler();
     EventHandler eventHandler = dispatcher.getEventHandler();
     eventHandler.handle(new ApplicationContainerFinishedEvent(containerID));
     eventHandler.handle(new ApplicationContainerFinishedEvent(containerID));
@@ -475,7 +489,7 @@ public class ContainerImpl implements Container {
     @Override
     @Override
     public ContainerState transition(ContainerImpl container,
     public ContainerState transition(ContainerImpl container,
         ContainerEvent event) {
         ContainerEvent event) {
-      final ContainerLaunchContext ctxt = container.getLaunchContext();
+      final ContainerLaunchContext ctxt = container.launchContext;
       container.metrics.initingContainer();
       container.metrics.initingContainer();
 
 
       // Inform the AuxServices about the opaque serviceData
       // Inform the AuxServices about the opaque serviceData
@@ -486,9 +500,9 @@ public class ContainerImpl implements Container {
         for (Map.Entry<String,ByteBuffer> service : csd.entrySet()) {
         for (Map.Entry<String,ByteBuffer> service : csd.entrySet()) {
           container.dispatcher.getEventHandler().handle(
           container.dispatcher.getEventHandler().handle(
               new AuxServicesEvent(AuxServicesEventType.APPLICATION_INIT,
               new AuxServicesEvent(AuxServicesEventType.APPLICATION_INIT,
-                ctxt.getUser(), 
-                ctxt.getContainerId().getApplicationAttemptId().getApplicationId(),
-                service.getKey().toString(), service.getValue()));
+                  ctxt.getUser(), container.container.getId()
+                      .getApplicationAttemptId().getApplicationId(),
+                  service.getKey().toString(), service.getValue()));
         }
         }
       }
       }
 
 
@@ -571,7 +585,7 @@ public class ContainerImpl implements Container {
           container.pendingResources.remove(rsrcEvent.getResource());
           container.pendingResources.remove(rsrcEvent.getResource());
       if (null == syms) {
       if (null == syms) {
         LOG.warn("Localized unknown resource " + rsrcEvent.getResource() +
         LOG.warn("Localized unknown resource " + rsrcEvent.getResource() +
-                 " for container " + container.getContainerID());
+                 " for container " + container.container.getId());
         assert false;
         assert false;
         // fail container?
         // fail container?
         return ContainerState.LOCALIZING;
         return ContainerState.LOCALIZING;
@@ -599,14 +613,14 @@ public class ContainerImpl implements Container {
       // Inform the ContainersMonitor to start monitoring the container's
       // Inform the ContainersMonitor to start monitoring the container's
       // resource usage.
       // resource usage.
       long pmemBytes =
       long pmemBytes =
-          container.getLaunchContext().getResource().getMemory() * 1024 * 1024L;
+          container.container.getResource().getMemory() * 1024 * 1024L;
       float pmemRatio = container.daemonConf.getFloat(
       float pmemRatio = container.daemonConf.getFloat(
           YarnConfiguration.NM_VMEM_PMEM_RATIO,
           YarnConfiguration.NM_VMEM_PMEM_RATIO,
           YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
           YarnConfiguration.DEFAULT_NM_VMEM_PMEM_RATIO);
       long vmemBytes = (long) (pmemRatio * pmemBytes);
       long vmemBytes = (long) (pmemRatio * pmemBytes);
       
       
       container.dispatcher.getEventHandler().handle(
       container.dispatcher.getEventHandler().handle(
-          new ContainerStartMonitoringEvent(container.getContainerID(),
+          new ContainerStartMonitoringEvent(container.container.getId(),
               vmemBytes, pmemBytes));
               vmemBytes, pmemBytes));
       container.metrics.runningContainer();
       container.metrics.runningContainer();
     }
     }
@@ -740,7 +754,7 @@ public class ContainerImpl implements Container {
           container.pendingResources.remove(rsrcEvent.getResource());
           container.pendingResources.remove(rsrcEvent.getResource());
       if (null == syms) {
       if (null == syms) {
         LOG.warn("Localized unknown resource " + rsrcEvent.getResource() +
         LOG.warn("Localized unknown resource " + rsrcEvent.getResource() +
-                 " for container " + container.getContainerID());
+                 " for container " + container.container.getId());
         assert false;
         assert false;
         // fail container?
         // fail container?
         return;
         return;
@@ -845,10 +859,9 @@ public class ContainerImpl implements Container {
   public String toString() {
   public String toString() {
     this.readLock.lock();
     this.readLock.lock();
     try {
     try {
-      return ConverterUtils.toString(launchContext.getContainerId());
+      return ConverterUtils.toString(container.getId());
     } finally {
     } finally {
       this.readLock.unlock();
       this.readLock.unlock();
     }
     }
   }
   }
-
 }
 }

+ 7 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/ContainerLaunch.java

@@ -246,9 +246,8 @@ public class ContainerLaunch implements Callable<Integer> {
     } catch (Throwable e) {
     } catch (Throwable e) {
       LOG.warn("Failed to launch container.", e);
       LOG.warn("Failed to launch container.", e);
       dispatcher.getEventHandler().handle(new ContainerExitEvent(
       dispatcher.getEventHandler().handle(new ContainerExitEvent(
-            launchContext.getContainerId(),
-            ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, ret,
-            e.getMessage()));
+          containerID, ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, ret,
+          e.getMessage()));
       return ret;
       return ret;
     } finally {
     } finally {
       completed.set(true);
       completed.set(true);
@@ -264,7 +263,7 @@ public class ContainerLaunch implements Callable<Integer> {
       // If the process was killed, Send container_cleanedup_after_kill and
       // If the process was killed, Send container_cleanedup_after_kill and
       // just break out of this method.
       // just break out of this method.
       dispatcher.getEventHandler().handle(
       dispatcher.getEventHandler().handle(
-            new ContainerExitEvent(launchContext.getContainerId(),
+            new ContainerExitEvent(containerID,
                 ContainerEventType.CONTAINER_KILLED_ON_REQUEST, ret,
                 ContainerEventType.CONTAINER_KILLED_ON_REQUEST, ret,
                 "Container exited with a non-zero exit code " + ret));
                 "Container exited with a non-zero exit code " + ret));
       return ret;
       return ret;
@@ -273,15 +272,15 @@ public class ContainerLaunch implements Callable<Integer> {
     if (ret != 0) {
     if (ret != 0) {
       LOG.warn("Container exited with a non-zero exit code " + ret);
       LOG.warn("Container exited with a non-zero exit code " + ret);
       this.dispatcher.getEventHandler().handle(new ContainerExitEvent(
       this.dispatcher.getEventHandler().handle(new ContainerExitEvent(
-              launchContext.getContainerId(),
-              ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, ret,
-              "Container exited with a non-zero exit code " + ret));
+          containerID,
+          ContainerEventType.CONTAINER_EXITED_WITH_FAILURE, ret,
+          "Container exited with a non-zero exit code " + ret));
       return ret;
       return ret;
     }
     }
 
 
     LOG.info("Container " + containerIdStr + " succeeded ");
     LOG.info("Container " + containerIdStr + " succeeded ");
     dispatcher.getEventHandler().handle(
     dispatcher.getEventHandler().handle(
-        new ContainerEvent(launchContext.getContainerId(),
+        new ContainerEvent(containerID,
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS));
             ContainerEventType.CONTAINER_EXITED_WITH_SUCCESS));
     return 0;
     return 0;
   }
   }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/webapp/dao/ContainerInfo.java

@@ -72,7 +72,7 @@ public class ContainerInfo {
     }
     }
 
 
     this.user = container.getUser();
     this.user = container.getUser();
-    Resource res = container.getLaunchContext().getResource();
+    Resource res = container.getResource();
     if (res != null) {
     if (res != null) {
       this.totalMemoryNeededMB = res.getMemory();
       this.totalMemoryNeededMB = res.getMemory();
     }
     }

+ 8 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java

@@ -27,6 +27,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 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.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -43,6 +44,8 @@ import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.nodemanager.security.NMContainerTokenSecretManager;
 import org.apache.hadoop.yarn.server.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 {
 
 
@@ -117,12 +120,15 @@ public class TestEventFlow {
     applicationAttemptId.setApplicationId(applicationId);
     applicationAttemptId.setApplicationId(applicationId);
     applicationAttemptId.setAttemptId(0);
     applicationAttemptId.setAttemptId(0);
     cID.setApplicationAttemptId(applicationAttemptId);
     cID.setApplicationAttemptId(applicationAttemptId);
-    launchContext.setContainerId(cID);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getId()).thenReturn(cID);
+    when(mockContainer.getResource()).thenReturn(recordFactory
+        .newRecordInstance(Resource.class));
     launchContext.setUser("testing");
     launchContext.setUser("testing");
-    launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
     StartContainerRequest request = 
     StartContainerRequest request = 
         recordFactory.newRecordInstance(StartContainerRequest.class);
         recordFactory.newRecordInstance(StartContainerRequest.class);
     request.setContainerLaunchContext(launchContext);
     request.setContainerLaunchContext(launchContext);
+    request.setContainer(mockContainer);
     containerManager.startContainer(request);
     containerManager.startContainer(request);
 
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,

+ 7 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java

@@ -99,7 +99,9 @@ public class TestNodeManagerReboot {
         Records.newRecord(ContainerLaunchContext.class);
         Records.newRecord(ContainerLaunchContext.class);
     // Construct the Container-id
     // Construct the Container-id
     ContainerId cId = createContainerId();
     ContainerId cId = createContainerId();
-    containerLaunchContext.setContainerId(cId);
+    org.apache.hadoop.yarn.api.records.Container mockContainer =
+        mock(org.apache.hadoop.yarn.api.records.Container.class);
+    when(mockContainer.getId()).thenReturn(cId);
 
 
     containerLaunchContext.setUser(user);
     containerLaunchContext.setUser(user);
 
 
@@ -122,12 +124,13 @@ public class TestNodeManagerReboot {
     containerLaunchContext.setUser(containerLaunchContext.getUser());
     containerLaunchContext.setUser(containerLaunchContext.getUser());
     List<String> commands = new ArrayList<String>();
     List<String> commands = new ArrayList<String>();
     containerLaunchContext.setCommands(commands);
     containerLaunchContext.setCommands(commands);
-    containerLaunchContext.setResource(Records
-        .newRecord(Resource.class));
-    containerLaunchContext.getResource().setMemory(1024);
+    Resource resource = Records.newRecord(Resource.class);
+    resource.setMemory(1024);
+    when(mockContainer.getResource()).thenReturn(resource);
     StartContainerRequest startRequest =
     StartContainerRequest startRequest =
         Records.newRecord(StartContainerRequest.class);
         Records.newRecord(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
+    startRequest.setContainer(mockContainer);
     containerManager.startContainer(startRequest);
     containerManager.startContainer(startRequest);
 
 
     GetContainerStatusRequest request =
     GetContainerStatusRequest request =

+ 11 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java

@@ -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.BufferedReader;
 import java.io.BufferedReader;
 import java.io.BufferedWriter;
 import java.io.BufferedWriter;
 import java.io.File;
 import java.io.File;
@@ -41,6 +44,7 @@ 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;
 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.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -56,8 +60,8 @@ 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.server.nodemanager.containermanager.ContainerManagerImpl;
-import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
 import org.apache.hadoop.yarn.server.nodemanager.metrics.NodeManagerMetrics;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
@@ -157,10 +161,10 @@ public class TestNodeManagerShutdown {
     
     
     ContainerLaunchContext containerLaunchContext =
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
+    Container mockContainer = mock(Container.class);
     // Construct the Container-id
     // Construct the Container-id
     ContainerId cId = createContainerId();
     ContainerId cId = createContainerId();
-    containerLaunchContext.setContainerId(cId);
+    when(mockContainer.getId()).thenReturn(cId);
 
 
     containerLaunchContext.setUser(user);
     containerLaunchContext.setUser(user);
 
 
@@ -184,12 +188,12 @@ public class TestNodeManagerShutdown {
     commands.add("/bin/bash");
     commands.add("/bin/bash");
     commands.add(scriptFile.getAbsolutePath());
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     containerLaunchContext.setCommands(commands);
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
-    containerLaunchContext.getResource().setMemory(1024);
+    Resource resource = BuilderUtils.newResource(1024, 1);
+    when(mockContainer.getResource()).thenReturn(resource);
     StartContainerRequest startRequest =
     StartContainerRequest startRequest =
         recordFactory.newRecordInstance(StartContainerRequest.class);
         recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
+    startRequest.setContainer(mockContainer);
     containerManager.startContainer(startRequest);
     containerManager.startContainer(startRequest);
     
     
     GetContainerStatusRequest request =
     GetContainerStatusRequest request =
@@ -287,7 +291,7 @@ public class TestNodeManagerShutdown {
 
 
       @Override
       @Override
       protected void rebootNodeStatusUpdater() {
       protected void rebootNodeStatusUpdater() {
-        ConcurrentMap<ContainerId, Container> containers =
+        ConcurrentMap<ContainerId, org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container> containers =
             getNMContext().getContainers();
             getNMContext().getContainers();
         // ensure that containers are empty before restart nodeStatusUpdater
         // ensure that containers are empty before restart nodeStatusUpdater
         Assert.assertTrue(containers.isEmpty());
         Assert.assertTrue(containers.isEmpty());

+ 14 - 10
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -176,6 +176,8 @@ public class TestNodeStatusUpdater {
       nodeStatus.setResponseId(heartBeatID++);
       nodeStatus.setResponseId(heartBeatID++);
       Map<ApplicationId, List<ContainerStatus>> appToContainers =
       Map<ApplicationId, List<ContainerStatus>> appToContainers =
           getAppToContainerStatusMap(nodeStatus.getContainersStatuses());
           getAppToContainerStatusMap(nodeStatus.getContainersStatuses());
+      org.apache.hadoop.yarn.api.records.Container mockContainer =
+          mock(org.apache.hadoop.yarn.api.records.Container.class);
       if (heartBeatID == 1) {
       if (heartBeatID == 1) {
         Assert.assertEquals(0, nodeStatus.getContainersStatuses().size());
         Assert.assertEquals(0, nodeStatus.getContainersStatuses().size());
 
 
@@ -186,11 +188,12 @@ public class TestNodeStatusUpdater {
         firstContainerID.setId(heartBeatID);
         firstContainerID.setId(heartBeatID);
         ContainerLaunchContext launchContext = recordFactory
         ContainerLaunchContext launchContext = recordFactory
             .newRecordInstance(ContainerLaunchContext.class);
             .newRecordInstance(ContainerLaunchContext.class);
-        launchContext.setContainerId(firstContainerID);
-        launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
-        launchContext.getResource().setMemory(2);
-        Container container = new ContainerImpl(conf , mockDispatcher,
-            launchContext, null, mockMetrics);
+        when(mockContainer.getId()).thenReturn(firstContainerID);
+        Resource resource = BuilderUtils.newResource(2, 1);
+        when(mockContainer.getResource()).thenReturn(resource);
+        Container container =
+            new ContainerImpl(conf, mockDispatcher, launchContext,
+                mockContainer, null, mockMetrics);
         this.context.getContainers().put(firstContainerID, container);
         this.context.getContainers().put(firstContainerID, container);
       } else if (heartBeatID == 2) {
       } else if (heartBeatID == 2) {
         // Checks on the RM end
         // Checks on the RM end
@@ -211,11 +214,12 @@ public class TestNodeStatusUpdater {
         secondContainerID.setId(heartBeatID);
         secondContainerID.setId(heartBeatID);
         ContainerLaunchContext launchContext = recordFactory
         ContainerLaunchContext launchContext = recordFactory
             .newRecordInstance(ContainerLaunchContext.class);
             .newRecordInstance(ContainerLaunchContext.class);
-        launchContext.setContainerId(secondContainerID);
-        launchContext.setResource(recordFactory.newRecordInstance(Resource.class));
-        launchContext.getResource().setMemory(3);
-        Container container = new ContainerImpl(conf, mockDispatcher,
-            launchContext, null, mockMetrics);
+        when(mockContainer.getId()).thenReturn(secondContainerID);
+        Resource resource = BuilderUtils.newResource(3, 1);
+        when(mockContainer.getResource()).thenReturn(resource);
+        Container container =
+            new ContainerImpl(conf, mockDispatcher, launchContext,
+                mockContainer, null, mockMetrics);
         this.context.getContainers().put(secondContainerID, container);
         this.context.getContainers().put(secondContainerID, container);
       } else if (heartBeatID == 3) {
       } else if (heartBeatID == 3) {
         // Checks on the RM end
         // Checks on the RM end

+ 24 - 18
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java

@@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 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.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -47,7 +48,6 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.exceptions.YarnRemoteException;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
 import org.apache.hadoop.yarn.server.nodemanager.CMgrCompletedAppsEvent;
@@ -58,8 +58,10 @@ import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.Ap
 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.server.security.ApplicationACLsManager;
+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;
+import static org.mockito.Mockito.*;
 
 
 public class TestContainerManager extends BaseContainerManagerTest {
 public class TestContainerManager extends BaseContainerManagerTest {
 
 
@@ -121,7 +123,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
 
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ContainerId cId = createContainerId();
     ContainerId cId = createContainerId();
-    container.setContainerId(cId);
 
 
     container.setUser(user);
     container.setUser(user);
 
 
@@ -143,14 +144,16 @@ public class TestContainerManager extends BaseContainerManagerTest {
     localResources.put(destinationFile, rsrc_alpha);
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);
     containerLaunchContext.setLocalResources(localResources);
     containerLaunchContext.setUser(container.getUser());
     containerLaunchContext.setUser(container.getUser());
-    containerLaunchContext.setContainerId(container.getContainerId());
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getId()).thenReturn(cId);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(512, 1));
 
 
     StartContainerRequest startRequest = 
     StartContainerRequest startRequest = 
         recordFactory.newRecordInstance(StartContainerRequest.class);
         recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
-    
+    startRequest.setContainer(mockContainer);
+
     containerManager.startContainer(startRequest);
     containerManager.startContainer(startRequest);
 
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
@@ -208,7 +211,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
 
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ContainerId cId = createContainerId();
     ContainerId cId = createContainerId();
-    containerLaunchContext.setContainerId(cId);
 
 
     containerLaunchContext.setUser(user);
     containerLaunchContext.setUser(user);
 
 
@@ -232,11 +234,13 @@ public class TestContainerManager extends BaseContainerManagerTest {
     commands.add("/bin/bash");
     commands.add("/bin/bash");
     commands.add(scriptFile.getAbsolutePath());
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     containerLaunchContext.setCommands(commands);
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
-    containerLaunchContext.getResource().setMemory(100 * 1024 * 1024);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getId()).thenReturn(cId);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(100 * 1024 * 1024, 1));
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
+    startRequest.setContainer(mockContainer);
     containerManager.startContainer(startRequest);
     containerManager.startContainer(startRequest);
  
  
     int timeoutSecs = 0;
     int timeoutSecs = 0;
@@ -310,7 +314,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
 
 	  // ////// Construct the Container-id
 	  // ////// Construct the Container-id
 	  ContainerId cId = createContainerId();
 	  ContainerId cId = createContainerId();
-	  containerLaunchContext.setContainerId(cId);
 
 
 	  containerLaunchContext.setUser(user);
 	  containerLaunchContext.setUser(user);
 
 
@@ -334,12 +337,14 @@ public class TestContainerManager extends BaseContainerManagerTest {
 	  commands.add("/bin/bash");
 	  commands.add("/bin/bash");
 	  commands.add(scriptFile.getAbsolutePath());
 	  commands.add(scriptFile.getAbsolutePath());
 	  containerLaunchContext.setCommands(commands);
 	  containerLaunchContext.setCommands(commands);
-	  containerLaunchContext.setResource(recordFactory
-			  .newRecordInstance(Resource.class));
-	  containerLaunchContext.getResource().setMemory(100 * 1024 * 1024);
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getId()).thenReturn(cId);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(100 * 1024 * 1024, 1));
 
 
 	  StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
 	  StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
 	  startRequest.setContainerLaunchContext(containerLaunchContext);
 	  startRequest.setContainerLaunchContext(containerLaunchContext);
+	  startRequest.setContainer(mockContainer);
 	  containerManager.startContainer(startRequest);
 	  containerManager.startContainer(startRequest);
 
 
 	  BaseContainerManagerTest.waitForContainerState(containerManager, cId,
 	  BaseContainerManagerTest.waitForContainerState(containerManager, cId,
@@ -402,7 +407,6 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ContainerId cId = createContainerId();
     ContainerId cId = createContainerId();
     ApplicationId appId = cId.getApplicationAttemptId().getApplicationId();
     ApplicationId appId = cId.getApplicationAttemptId().getApplicationId();
-    container.setContainerId(cId);
 
 
     container.setUser(user);
     container.setUser(user);
 
 
@@ -425,14 +429,16 @@ public class TestContainerManager extends BaseContainerManagerTest {
     localResources.put(destinationFile, rsrc_alpha);
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);
     containerLaunchContext.setLocalResources(localResources);
     containerLaunchContext.setUser(container.getUser());
     containerLaunchContext.setUser(container.getUser());
-    containerLaunchContext.setContainerId(container.getContainerId());
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
+    Container mockContainer = mock(Container.class);
+    when(mockContainer.getId()).thenReturn(cId);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(100, 1));
 
 
 //    containerLaunchContext.command = new ArrayList<CharSequence>();
 //    containerLaunchContext.command = new ArrayList<CharSequence>();
 
 
     StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
     StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
     request.setContainerLaunchContext(containerLaunchContext);
     request.setContainerLaunchContext(containerLaunchContext);
+    request.setContainer(mockContainer);
     containerManager.startContainer(request);
     containerManager.startContainer(request);
 
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,

+ 8 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/container/TestContainer.java

@@ -525,8 +525,9 @@ public class TestContainer {
     return serviceData;
     return serviceData;
   }
   }
 
 
-  private Container newContainer(Dispatcher disp, ContainerLaunchContext ctx) {
-    return new ContainerImpl(conf, disp, ctx, null, metrics);
+  private Container newContainer(Dispatcher disp, ContainerLaunchContext ctx,
+      org.apache.hadoop.yarn.api.records.Container container) {
+    return new ContainerImpl(conf, disp, ctx, container, null, metrics);
   }
   }
   
   
   @SuppressWarnings("unchecked")
   @SuppressWarnings("unchecked")
@@ -570,12 +571,14 @@ public class TestContainer {
       this.user = user;
       this.user = user;
 
 
       ctxt = mock(ContainerLaunchContext.class);
       ctxt = mock(ContainerLaunchContext.class);
+      org.apache.hadoop.yarn.api.records.Container mockContainer =
+          mock(org.apache.hadoop.yarn.api.records.Container.class);
       cId = BuilderUtils.newContainerId(appId, 1, timestamp, id);
       cId = BuilderUtils.newContainerId(appId, 1, timestamp, id);
       when(ctxt.getUser()).thenReturn(this.user);
       when(ctxt.getUser()).thenReturn(this.user);
-      when(ctxt.getContainerId()).thenReturn(cId);
+      when(mockContainer.getId()).thenReturn(cId);
 
 
       Resource resource = BuilderUtils.newResource(1024, 1);
       Resource resource = BuilderUtils.newResource(1024, 1);
-      when(ctxt.getResource()).thenReturn(resource);
+      when(mockContainer.getResource()).thenReturn(resource);
 
 
       if (withLocalRes) {
       if (withLocalRes) {
         Random r = new Random();
         Random r = new Random();
@@ -599,7 +602,7 @@ public class TestContainer {
       }
       }
       when(ctxt.getServiceData()).thenReturn(serviceData);
       when(ctxt.getServiceData()).thenReturn(serviceData);
 
 
-      c = newContainer(dispatcher, ctxt);
+      c = newContainer(dispatcher, ctxt, mockContainer);
       dispatcher.start();
       dispatcher.start();
     }
     }
 
 

+ 14 - 9
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/launcher/TestContainerLaunch.java

@@ -43,6 +43,7 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
 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.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -57,11 +58,14 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch;
+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.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ResourceCalculatorPlugin;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
+import static org.mockito.Mockito.*;
+
 import junit.framework.Assert;
 import junit.framework.Assert;
 
 
 public class TestContainerLaunch extends BaseContainerManagerTest {
 public class TestContainerLaunch extends BaseContainerManagerTest {
@@ -184,6 +188,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     ContainerLaunchContext containerLaunchContext = 
     ContainerLaunchContext containerLaunchContext = 
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
 
+    Container mockContainer = mock(Container.class);
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
     appId.setClusterTimestamp(0);
     appId.setClusterTimestamp(0);
@@ -195,7 +200,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     ContainerId cId = 
     ContainerId cId = 
         recordFactory.newRecordInstance(ContainerId.class);
         recordFactory.newRecordInstance(ContainerId.class);
     cId.setApplicationAttemptId(appAttemptId);
     cId.setApplicationAttemptId(appAttemptId);
-    containerLaunchContext.setContainerId(cId);
+    when(mockContainer.getId()).thenReturn(cId);
 
 
     containerLaunchContext.setUser(user);
     containerLaunchContext.setUser(user);
 
 
@@ -222,11 +227,11 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     commands.add("/bin/bash");
     commands.add("/bin/bash");
     commands.add(scriptFile.getAbsolutePath());
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     containerLaunchContext.setCommands(commands);
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
-    containerLaunchContext.getResource().setMemory(1024);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(1024, 1));
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
+    startRequest.setContainer(mockContainer);
     containerManager.startContainer(startRequest);
     containerManager.startContainer(startRequest);
 
 
     int timeoutSecs = 0;
     int timeoutSecs = 0;
@@ -301,7 +306,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
 
 
     ContainerLaunchContext containerLaunchContext = 
     ContainerLaunchContext containerLaunchContext = 
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
+    Container mockContainer = mock(Container.class);
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
     appId.setClusterTimestamp(1);
     appId.setClusterTimestamp(1);
@@ -313,7 +318,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     ContainerId cId = 
     ContainerId cId = 
         recordFactory.newRecordInstance(ContainerId.class);
         recordFactory.newRecordInstance(ContainerId.class);
     cId.setApplicationAttemptId(appAttemptId);
     cId.setApplicationAttemptId(appAttemptId);
-    containerLaunchContext.setContainerId(cId);
+    when(mockContainer.getId()).thenReturn(cId);
 
 
     containerLaunchContext.setUser(user);
     containerLaunchContext.setUser(user);
 
 
@@ -339,11 +344,11 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     List<String> commands = new ArrayList<String>();
     List<String> commands = new ArrayList<String>();
     commands.add(scriptFile.getAbsolutePath());
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     containerLaunchContext.setCommands(commands);
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
-    containerLaunchContext.getResource().setMemory(1024);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(1024, 1));
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
+    startRequest.setContainer(mockContainer);
     containerManager.startContainer(startRequest);
     containerManager.startContainer(startRequest);
 
 
     int timeoutSecs = 0;
     int timeoutSecs = 0;

+ 7 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/TestLogAggregationService.java

@@ -56,13 +56,13 @@ import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
 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.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 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.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.DrainDispatcher;
 import org.apache.hadoop.yarn.event.DrainDispatcher;
@@ -91,6 +91,7 @@ import org.mockito.Mockito;
 import org.mortbay.util.MultiException;
 import org.mortbay.util.MultiException;
 
 
 
 
+
 //@Ignore
 //@Ignore
 public class TestLogAggregationService extends BaseContainerManagerTest {
 public class TestLogAggregationService extends BaseContainerManagerTest {
 
 
@@ -679,7 +680,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
 
 
     ContainerLaunchContext containerLaunchContext =
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
+    Container mockContainer = mock(Container.class);
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ApplicationId appId =
     ApplicationId appId =
         recordFactory.newRecordInstance(ApplicationId.class);
         recordFactory.newRecordInstance(ApplicationId.class);
@@ -689,7 +690,7 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
         BuilderUtils.newApplicationAttemptId(appId, 1);
         BuilderUtils.newApplicationAttemptId(appId, 1);
     ContainerId cId = BuilderUtils.newContainerId(appAttemptId, 0);
     ContainerId cId = BuilderUtils.newContainerId(appAttemptId, 0);
 
 
-    containerLaunchContext.setContainerId(cId);
+    when(mockContainer.getId()).thenReturn(cId);
 
 
     containerLaunchContext.setUser(this.user);
     containerLaunchContext.setUser(this.user);
 
 
@@ -713,12 +714,12 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     commands.add("/bin/bash");
     commands.add("/bin/bash");
     commands.add(scriptFile.getAbsolutePath());
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     containerLaunchContext.setCommands(commands);
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
-    containerLaunchContext.getResource().setMemory(100 * 1024 * 1024);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(100 * 1024 * 1024, 1));
     StartContainerRequest startRequest =
     StartContainerRequest startRequest =
         recordFactory.newRecordInstance(StartContainerRequest.class);
         recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
+    startRequest.setContainer(mockContainer);
     this.containerManager.startContainer(startRequest);
     this.containerManager.startContainer(startRequest);
 
 
     BaseContainerManagerTest.waitForContainerState(this.containerManager,
     BaseContainerManagerTest.waitForContainerState(this.containerManager,

+ 8 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/monitor/TestContainersMonitor.java

@@ -21,7 +21,7 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager.monitor;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertFalse;
 import static org.junit.Assert.assertTrue;
 import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.*;
 
 
 import java.io.BufferedReader;
 import java.io.BufferedReader;
 import java.io.File;
 import java.io.File;
@@ -44,6 +44,7 @@ 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;
 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.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -51,7 +52,6 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
-import org.apache.hadoop.yarn.api.records.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.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -60,6 +60,7 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.Signal;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.Context;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.BaseContainerManagerTest;
+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.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.LinuxResourceCalculatorPlugin;
 import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
 import org.apache.hadoop.yarn.util.ProcfsBasedProcessTree;
@@ -197,7 +198,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
 
 
     ContainerLaunchContext containerLaunchContext =
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
-
+    Container mockContainer = mock(Container.class);
     // ////// Construct the Container-id
     // ////// Construct the Container-id
     ApplicationId appId =
     ApplicationId appId =
         recordFactory.newRecordInstance(ApplicationId.class);
         recordFactory.newRecordInstance(ApplicationId.class);
@@ -210,7 +211,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
     ContainerId cId = recordFactory.newRecordInstance(ContainerId.class);
     cId.setId(0);
     cId.setId(0);
     cId.setApplicationAttemptId(appAttemptId);
     cId.setApplicationAttemptId(appAttemptId);
-    containerLaunchContext.setContainerId(cId);
+    when(mockContainer.getId()).thenReturn(cId);
 
 
     containerLaunchContext.setUser(user);
     containerLaunchContext.setUser(user);
 
 
@@ -234,12 +235,12 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     commands.add("/bin/bash");
     commands.add("/bin/bash");
     commands.add(scriptFile.getAbsolutePath());
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     containerLaunchContext.setCommands(commands);
-    containerLaunchContext.setResource(recordFactory
-        .newRecordInstance(Resource.class));
-    containerLaunchContext.getResource().setMemory(8 * 1024 * 1024);
+    when(mockContainer.getResource()).thenReturn(
+        BuilderUtils.newResource(8 * 1024 * 1024, 1));
     StartContainerRequest startRequest =
     StartContainerRequest startRequest =
         recordFactory.newRecordInstance(StartContainerRequest.class);
         recordFactory.newRecordInstance(StartContainerRequest.class);
     startRequest.setContainerLaunchContext(containerLaunchContext);
     startRequest.setContainerLaunchContext(containerLaunchContext);
+    startRequest.setContainer(mockContainer);
     containerManager.startContainer(startRequest);
     containerManager.startContainer(startRequest);
 
 
     int timeoutSecs = 0;
     int timeoutSecs = 0;

+ 6 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/MockContainer.java

@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
 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;
@@ -58,7 +59,6 @@ public class MockContainer implements Container {
         uniqId);
         uniqId);
     this.launchContext = recordFactory
     this.launchContext = recordFactory
         .newRecordInstance(ContainerLaunchContext.class);
         .newRecordInstance(ContainerLaunchContext.class);
-    launchContext.setContainerId(id);
     launchContext.setUser(user);
     launchContext.setUser(user);
     this.state = ContainerState.NEW;
     this.state = ContainerState.NEW;
 
 
@@ -104,7 +104,6 @@ public class MockContainer implements Container {
         .newRecordInstance(ContainerStatus.class);
         .newRecordInstance(ContainerStatus.class);
     containerStatus
     containerStatus
         .setState(org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
         .setState(org.apache.hadoop.yarn.api.records.ContainerState.RUNNING);
-    containerStatus.setContainerId(this.launchContext.getContainerId());
     containerStatus.setDiagnostics("testing");
     containerStatus.setDiagnostics("testing");
     containerStatus.setExitStatus(0);
     containerStatus.setExitStatus(0);
     return containerStatus;
     return containerStatus;
@@ -119,4 +118,9 @@ public class MockContainer implements Container {
   public void handle(ContainerEvent event) {
   public void handle(ContainerEvent event) {
   }
   }
 
 
+  @Override
+  public Resource getResource() {
+    return null;
+  }
+
 }
 }

+ 11 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/webapp/TestNMWebServer.java

@@ -179,15 +179,19 @@ public class TestNMWebServer {
       // TODO: Use builder utils
       // TODO: Use builder utils
       ContainerLaunchContext launchContext =
       ContainerLaunchContext launchContext =
           recordFactory.newRecordInstance(ContainerLaunchContext.class);
           recordFactory.newRecordInstance(ContainerLaunchContext.class);
-      launchContext.setContainerId(containerId);
+      org.apache.hadoop.yarn.api.records.Container mockContainer =
+          mock(org.apache.hadoop.yarn.api.records.Container.class);
+      when(mockContainer.getId()).thenReturn(containerId);
       launchContext.setUser(user);
       launchContext.setUser(user);
       Container container =
       Container container =
-          new ContainerImpl(conf, dispatcher, launchContext, null, metrics) {
-        @Override
-        public ContainerState getContainerState() {
-          return ContainerState.RUNNING;
-        };
-      };
+          new ContainerImpl(conf, dispatcher, launchContext, mockContainer,
+              null, metrics) {
+
+            @Override
+            public ContainerState getContainerState() {
+              return ContainerState.RUNNING;
+            };
+          };
       nmContext.getContainers().put(containerId, container);
       nmContext.getContainers().put(containerId, container);
       //TODO: Gross hack. Fix in code.
       //TODO: Gross hack. Fix in code.
       ApplicationId applicationId = 
       ApplicationId applicationId = 

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/ClientRMService.java

@@ -266,7 +266,7 @@ public class ClientRMService extends AbstractService implements
     ApplicationSubmissionContext submissionContext = request
     ApplicationSubmissionContext submissionContext = request
         .getApplicationSubmissionContext();
         .getApplicationSubmissionContext();
     ApplicationId applicationId = submissionContext.getApplicationId();
     ApplicationId applicationId = submissionContext.getApplicationId();
-    String user = submissionContext.getUser();
+    String user = submissionContext.getAMContainerSpec().getUser();
     try {
     try {
       user = UserGroupInformation.getCurrentUser().getShortUserName();
       user = UserGroupInformation.getCurrentUser().getShortUserName();
       if (rmContext.getRMApps().get(applicationId) != null) {
       if (rmContext.getRMApps().get(applicationId) != null) {
@@ -275,13 +275,13 @@ public class ClientRMService extends AbstractService implements
       }
       }
 
 
       // Safety 
       // Safety 
-      submissionContext.setUser(user);
+      submissionContext.getAMContainerSpec().setUser(user);
 
 
       // Check whether AM resource requirements are within required limits
       // Check whether AM resource requirements are within required limits
       if (!submissionContext.getUnmanagedAM()) {
       if (!submissionContext.getUnmanagedAM()) {
         ResourceRequest amReq = BuilderUtils.newResourceRequest(
         ResourceRequest amReq = BuilderUtils.newResourceRequest(
             RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
             RMAppAttemptImpl.AM_CONTAINER_PRIORITY, ResourceRequest.ANY,
-            submissionContext.getAMContainerSpec().getResource(), 1);
+            submissionContext.getResource(), 1);
         try {
         try {
           SchedulerUtils.validateResourceRequest(amReq,
           SchedulerUtils.validateResourceRequest(amReq,
               scheduler.getMaximumResourceCapability());
               scheduler.getMaximumResourceCapability());

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

@@ -249,10 +249,11 @@ public class RMAppManager implements EventHandler<RMAppManagerEvent>,
       // Create RMApp
       // Create RMApp
       application =
       application =
           new RMAppImpl(applicationId, rmContext, this.conf,
           new RMAppImpl(applicationId, rmContext, this.conf,
-            submissionContext.getApplicationName(),
-            submissionContext.getUser(), submissionContext.getQueue(),
-            submissionContext, this.scheduler, this.masterService,
-            submitTime);
+              submissionContext.getApplicationName(),
+              submissionContext.getAMContainerSpec().getUser(),
+              submissionContext.getQueue(),
+              submissionContext, this.scheduler, this.masterService,
+              submitTime);
 
 
       // Sanity check - duplicate?
       // Sanity check - duplicate?
       if (rmContext.getRMApps().putIfAbsent(applicationId, application) != 
       if (rmContext.getRMApps().putIfAbsent(applicationId, application) != 

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

@@ -77,6 +77,7 @@ public class AMLauncher implements Runnable {
       RecordFactoryProvider.getRecordFactory(null);
       RecordFactoryProvider.getRecordFactory(null);
   private final AMLauncherEventType eventType;
   private final AMLauncherEventType eventType;
   private final RMContext rmContext;
   private final RMContext rmContext;
+  private final Container masterContainer;
   
   
   @SuppressWarnings("rawtypes")
   @SuppressWarnings("rawtypes")
   private final EventHandler handler;
   private final EventHandler handler;
@@ -88,34 +89,36 @@ public class AMLauncher implements Runnable {
     this.eventType = eventType;
     this.eventType = eventType;
     this.rmContext = rmContext;
     this.rmContext = rmContext;
     this.handler = rmContext.getDispatcher().getEventHandler();
     this.handler = rmContext.getDispatcher().getEventHandler();
+    this.masterContainer = application.getMasterContainer();
   }
   }
   
   
   private void connect() throws IOException {
   private void connect() throws IOException {
-    ContainerId masterContainerID = application.getMasterContainer().getId();
+    ContainerId masterContainerID = masterContainer.getId();
     
     
     containerMgrProxy = getContainerMgrProxy(masterContainerID);
     containerMgrProxy = getContainerMgrProxy(masterContainerID);
   }
   }
   
   
   private void launch() throws IOException {
   private void launch() throws IOException {
     connect();
     connect();
-    ContainerId masterContainerID = application.getMasterContainer().getId();
+    ContainerId masterContainerID = masterContainer.getId();
     ApplicationSubmissionContext applicationContext =
     ApplicationSubmissionContext applicationContext =
       application.getSubmissionContext();
       application.getSubmissionContext();
-    LOG.info("Setting up container " + application.getMasterContainer() 
+    LOG.info("Setting up container " + masterContainer
         + " for AM " + application.getAppAttemptId());  
         + " for AM " + application.getAppAttemptId());  
     ContainerLaunchContext launchContext =
     ContainerLaunchContext launchContext =
         createAMContainerLaunchContext(applicationContext, masterContainerID);
         createAMContainerLaunchContext(applicationContext, masterContainerID);
     StartContainerRequest request = 
     StartContainerRequest request = 
         recordFactory.newRecordInstance(StartContainerRequest.class);
         recordFactory.newRecordInstance(StartContainerRequest.class);
     request.setContainerLaunchContext(launchContext);
     request.setContainerLaunchContext(launchContext);
+    request.setContainer(masterContainer);
     containerMgrProxy.startContainer(request);
     containerMgrProxy.startContainer(request);
-    LOG.info("Done launching container " + application.getMasterContainer() 
+    LOG.info("Done launching container " + masterContainer
         + " for AM " + application.getAppAttemptId());
         + " for AM " + application.getAppAttemptId());
   }
   }
   
   
   private void cleanup() throws IOException {
   private void cleanup() throws IOException {
     connect();
     connect();
-    ContainerId containerId = application.getMasterContainer().getId();
+    ContainerId containerId = masterContainer.getId();
     StopContainerRequest stopRequest = 
     StopContainerRequest stopRequest = 
         recordFactory.newRecordInstance(StopContainerRequest.class);
         recordFactory.newRecordInstance(StopContainerRequest.class);
     stopRequest.setContainerId(containerId);
     stopRequest.setContainerId(containerId);
@@ -126,9 +129,7 @@ public class AMLauncher implements Runnable {
   protected ContainerManager getContainerMgrProxy(
   protected ContainerManager getContainerMgrProxy(
       final ContainerId containerId) {
       final ContainerId containerId) {
 
 
-    Container container = application.getMasterContainer();
-
-    final NodeId node = container.getNodeId();
+    final NodeId node = masterContainer.getNodeId();
     final InetSocketAddress containerManagerBindAddress =
     final InetSocketAddress containerManagerBindAddress =
         NetUtils.createSocketAddrForHost(node.getHost(), node.getPort());
         NetUtils.createSocketAddrForHost(node.getHost(), node.getPort());
 
 
@@ -138,8 +139,8 @@ public class AMLauncher implements Runnable {
         .createRemoteUser(containerId.toString());
         .createRemoteUser(containerId.toString());
     if (UserGroupInformation.isSecurityEnabled()) {
     if (UserGroupInformation.isSecurityEnabled()) {
       Token<ContainerTokenIdentifier> token =
       Token<ContainerTokenIdentifier> token =
-          ProtoUtils.convertFromProtoFormat(container.getContainerToken(),
-                                            containerManagerBindAddress);
+          ProtoUtils.convertFromProtoFormat(masterContainer
+              .getContainerToken(), containerManagerBindAddress);
       currentUser.addToken(token);
       currentUser.addToken(token);
     }
     }
     return currentUser.doAs(new PrivilegedAction<ContainerManager>() {
     return currentUser.doAs(new PrivilegedAction<ContainerManager>() {
@@ -165,30 +166,28 @@ public class AMLauncher implements Runnable {
             new String[0])));
             new String[0])));
     
     
     // Finalize the container
     // Finalize the container
-    container.setContainerId(containerID);
-    container.setUser(applicationMasterContext.getUser());
-    setupTokensAndEnv(container);
+    container.setUser(applicationMasterContext.getAMContainerSpec().getUser());
+    setupTokensAndEnv(container, containerID);
     
     
     return container;
     return container;
   }
   }
 
 
   private void setupTokensAndEnv(
   private void setupTokensAndEnv(
-      ContainerLaunchContext container)
+      ContainerLaunchContext container, ContainerId containerID)
       throws IOException {
       throws IOException {
     Map<String, String> environment = container.getEnvironment();
     Map<String, String> environment = container.getEnvironment();
-
     environment.put(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV,
     environment.put(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV,
         application.getWebProxyBase());
         application.getWebProxyBase());
     // Set the AppAttemptId, containerId, NMHTTPAdress, AppSubmitTime to be
     // Set the AppAttemptId, containerId, NMHTTPAdress, AppSubmitTime to be
     // consumable by the AM.
     // consumable by the AM.
-    environment.put(ApplicationConstants.AM_CONTAINER_ID_ENV, container
-        .getContainerId().toString());
-    environment.put(ApplicationConstants.NM_HOST_ENV, application
-        .getMasterContainer().getNodeId().getHost());
+    environment.put(ApplicationConstants.AM_CONTAINER_ID_ENV,
+        containerID.toString());
+    environment.put(ApplicationConstants.NM_HOST_ENV, masterContainer
+        .getNodeId().getHost());
     environment.put(ApplicationConstants.NM_PORT_ENV,
     environment.put(ApplicationConstants.NM_PORT_ENV,
-        String.valueOf(application.getMasterContainer().getNodeId().getPort()));
+        String.valueOf(masterContainer.getNodeId().getPort()));
     String parts[] =
     String parts[] =
-        application.getMasterContainer().getNodeHttpAddress().split(":");
+        masterContainer.getNodeHttpAddress().split(":");
     environment.put(ApplicationConstants.NM_HTTP_PORT_ENV, parts[1]);
     environment.put(ApplicationConstants.NM_HTTP_PORT_ENV, parts[1]);
     ApplicationId applicationId =
     ApplicationId applicationId =
         application.getAppAttemptId().getApplicationId();
         application.getAppAttemptId().getApplicationId();

+ 6 - 9
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

@@ -690,7 +690,7 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       appAttempt.eventHandler.handle(
       appAttempt.eventHandler.handle(
           new AppAddedSchedulerEvent(appAttempt.applicationAttemptId,
           new AppAddedSchedulerEvent(appAttempt.applicationAttemptId,
               appAttempt.submissionContext.getQueue(),
               appAttempt.submissionContext.getQueue(),
-              appAttempt.submissionContext.getUser()));
+              appAttempt.submissionContext.getAMContainerSpec().getUser()));
     }
     }
   }
   }
 
 
@@ -736,9 +736,10 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
             RMAppEventType.APP_ACCEPTED));
             RMAppEventType.APP_ACCEPTED));
 
 
         // Request a container for the AM.
         // Request a container for the AM.
-        ResourceRequest request = BuilderUtils.newResourceRequest(
-            AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt.submissionContext
-                .getAMContainerSpec().getResource(), 1);
+        ResourceRequest request =
+            BuilderUtils.newResourceRequest(
+                AM_CONTAINER_PRIORITY, ResourceRequest.ANY, appAttempt
+                    .getSubmissionContext().getResource(), 1);
 
 
         // SchedulerUtils.validateResourceRequests is not necessary because
         // SchedulerUtils.validateResourceRequests is not necessary because
         // AM resource has been checked when submission
         // AM resource has been checked when submission
@@ -773,12 +774,8 @@ public class RMAppAttemptImpl implements RMAppAttempt, Recoverable {
       // Set the masterContainer
       // Set the masterContainer
       appAttempt.setMasterContainer(amContainerAllocation.getContainers().get(
       appAttempt.setMasterContainer(amContainerAllocation.getContainers().get(
                                                                            0));
                                                                            0));
-      // Updating CLC's resource is no longer necessary once YARN-486 is
-      // completed, because nothing from Container to CLC will be copied into
-      // CLC then.
-      appAttempt.getSubmissionContext().getAMContainerSpec().setResource(
+      appAttempt.getSubmissionContext().setResource(
           appAttempt.getMasterContainer().getResource());
           appAttempt.getMasterContainer().getResource());
-
       RMStateStore store = appAttempt.rmContext.getStateStore();
       RMStateStore store = appAttempt.rmContext.getStateStore();
       appAttempt.storeAttempt(store);
       appAttempt.storeAttempt(store);
     }
     }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/dao/AppAttemptInfo.java

@@ -59,8 +59,8 @@ public class AppAttemptInfo {
         this.logsLink = join(HttpConfig.getSchemePrefix(),
         this.logsLink = join(HttpConfig.getSchemePrefix(),
             masterContainer.getNodeHttpAddress(),
             masterContainer.getNodeHttpAddress(),
             "/node", "/containerlogs/",
             "/node", "/containerlogs/",
-            ConverterUtils.toString(masterContainer.getId()),
-            "/", attempt.getSubmissionContext().getUser());
+            ConverterUtils.toString(masterContainer.getId()), "/",
+            attempt.getSubmissionContext().getAMContainerSpec().getUser());
       }
       }
     }
     }
   }
   }

+ 4 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -130,7 +130,7 @@ public class Application {
   public synchronized void submit() throws IOException {
   public synchronized void submit() throws IOException {
     ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     ApplicationSubmissionContext context = recordFactory.newRecordInstance(ApplicationSubmissionContext.class);
     context.setApplicationId(this.applicationId);
     context.setApplicationId(this.applicationId);
-    context.setUser(this.user);
+    context.getAMContainerSpec().setUser(this.user);
     context.setQueue(this.queue);
     context.setQueue(this.queue);
     SubmitApplicationRequest request = recordFactory
     SubmitApplicationRequest request = recordFactory
         .newRecordInstance(SubmitApplicationRequest.class);
         .newRecordInstance(SubmitApplicationRequest.class);
@@ -340,7 +340,8 @@ public class Application {
 
 
             // Launch the container
             // Launch the container
             StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
             StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
-            startRequest.setContainerLaunchContext(createCLC(container));
+            startRequest.setContainerLaunchContext(createCLC());
+            startRequest.setContainer(container);
             nodeManager.startContainer(startRequest);
             nodeManager.startContainer(startRequest);
             break;
             break;
           }
           }
@@ -396,11 +397,9 @@ public class Application {
     }
     }
   }
   }
 
 
-  private ContainerLaunchContext createCLC(Container container) {
+  private ContainerLaunchContext createCLC() {
     ContainerLaunchContext clc = recordFactory.newRecordInstance(ContainerLaunchContext.class);
     ContainerLaunchContext clc = recordFactory.newRecordInstance(ContainerLaunchContext.class);
-    clc.setContainerId(container.getId());
     clc.setUser(this.user);
     clc.setUser(this.user);
-    clc.setResource(container.getResource());
     return clc;
     return clc;
   }
   }
 }
 }

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

@@ -161,7 +161,6 @@ public class MockRM extends ResourceManager {
         .newRecord(ApplicationSubmissionContext.class);
         .newRecord(ApplicationSubmissionContext.class);
     sub.setApplicationId(appId);
     sub.setApplicationId(appId);
     sub.setApplicationName(name);
     sub.setApplicationName(name);
-    sub.setUser(user);
     sub.setMaxAppAttempts(maxAppAttempts);
     sub.setMaxAppAttempts(maxAppAttempts);
     if(unmanaged) {
     if(unmanaged) {
       sub.setUnmanagedAM(true);
       sub.setUnmanagedAM(true);
@@ -171,13 +170,13 @@ public class MockRM extends ResourceManager {
     }
     }
     ContainerLaunchContext clc = Records
     ContainerLaunchContext clc = Records
         .newRecord(ContainerLaunchContext.class);
         .newRecord(ContainerLaunchContext.class);
-    Resource capability = Records.newRecord(Resource.class);
+    final Resource capability = Records.newRecord(Resource.class);
     capability.setMemory(masterMemory);
     capability.setMemory(masterMemory);
-    clc.setResource(capability);
+    sub.setResource(capability);
     clc.setApplicationACLs(acls);
     clc.setApplicationACLs(acls);
+    clc.setUser(user);
     sub.setAMContainerSpec(clc);
     sub.setAMContainerSpec(clc);
     req.setApplicationSubmissionContext(sub);
     req.setApplicationSubmissionContext(sub);
-
     UserGroupInformation fakeUser =
     UserGroupInformation fakeUser =
       UserGroupInformation.createUserForTesting(user, new String[] {"someGroup"});
       UserGroupInformation.createUserForTesting(user, new String[] {"someGroup"});
     PrivilegedAction<SubmitApplicationResponse> action =
     PrivilegedAction<SubmitApplicationResponse> action =

+ 9 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -40,7 +40,6 @@ import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
 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.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.api.records.NodeHealthStatus;
 import org.apache.hadoop.yarn.api.records.NodeHealthStatus;
@@ -162,11 +161,10 @@ public class NodeManager implements ContainerManager {
   synchronized public StartContainerResponse startContainer(
   synchronized public StartContainerResponse startContainer(
       StartContainerRequest request) 
       StartContainerRequest request) 
   throws YarnRemoteException {
   throws YarnRemoteException {
-    ContainerLaunchContext containerLaunchContext = 
-        request.getContainerLaunchContext();
-    
+    Container requestContainer = request.getContainer();
+
     ApplicationId applicationId = 
     ApplicationId applicationId = 
-        containerLaunchContext.getContainerId().getApplicationAttemptId().
+        requestContainer.getId().getApplicationAttemptId().
         getApplicationId();
         getApplicationId();
 
 
     List<Container> applicationContainers = containers.get(applicationId);
     List<Container> applicationContainers = containers.get(applicationId);
@@ -177,18 +175,18 @@ public class NodeManager implements ContainerManager {
     
     
     // Sanity check
     // Sanity check
     for (Container container : applicationContainers) {
     for (Container container : applicationContainers) {
-      if (container.getId().compareTo(containerLaunchContext.getContainerId()) 
+      if (container.getId().compareTo(requestContainer.getId())
           == 0) {
           == 0) {
         throw new IllegalStateException(
         throw new IllegalStateException(
-            "Container " + containerLaunchContext.getContainerId() + 
+            "Container " + requestContainer.getId() +
             " already setup on node " + containerManagerAddress);
             " already setup on node " + containerManagerAddress);
       }
       }
     }
     }
 
 
     Container container =
     Container container =
-        BuilderUtils.newContainer(containerLaunchContext.getContainerId(),
+        BuilderUtils.newContainer(requestContainer.getId(),
             this.nodeId, nodeHttpAddress,
             this.nodeId, nodeHttpAddress,
-            containerLaunchContext.getResource(), 
+            requestContainer.getResource(),
             null, null                                 // DKDC - Doesn't matter
             null, null                                 // DKDC - Doesn't matter
             );
             );
 
 
@@ -197,8 +195,8 @@ public class NodeManager implements ContainerManager {
             "", -1000);
             "", -1000);
     applicationContainers.add(container);
     applicationContainers.add(container);
     containerStatusMap.put(container, containerStatus);
     containerStatusMap.put(container, containerStatus);
-    Resources.subtractFrom(available, containerLaunchContext.getResource());
-    Resources.addTo(used, containerLaunchContext.getResource());
+    Resources.subtractFrom(available, requestContainer.getResource());
+    Resources.addTo(used, requestContainer.getResource());
     
     
     if(LOG.isDebugEnabled()) {
     if(LOG.isDebugEnabled()) {
       LOG.debug("startContainer:" + " node=" + containerManagerAddress
       LOG.debug("startContainer:" + " node=" + containerManagerAddress

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAppManager.java

@@ -50,6 +50,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.capacity.Capacity
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.resourcemanager.security.ClientToAMTokenSecretManagerInRM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
 import org.apache.hadoop.yarn.service.Service;
 import org.apache.hadoop.yarn.service.Service;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.Test;
 import org.junit.Test;
 
 
 import com.google.common.collect.Lists;
 import com.google.common.collect.Lists;
@@ -503,6 +504,10 @@ public class TestAppManager{
     RMApp appOrig = rmContext.getRMApps().get(appID);
     RMApp appOrig = rmContext.getRMApps().get(appID);
     Assert.assertTrue("app name matches but shouldn't", "testApp1" != appOrig.getName());
     Assert.assertTrue("app name matches but shouldn't", "testApp1" != appOrig.getName());
 
 
+    ContainerLaunchContext clc =
+        BuilderUtils.newContainerLaunchContext(null, null, null, null, null,
+            null, null);
+    context.setAMContainerSpec(clc);
     // our testApp1 should be rejected and original app with same id should be left in place
     // our testApp1 should be rejected and original app with same id should be left in place
     appMonitor.submitApplication(context);
     appMonitor.submitApplication(context);
 
 

+ 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

@@ -169,7 +169,7 @@ public class TestApplicationACLs {
     ContainerLaunchContext amContainer = recordFactory
     ContainerLaunchContext amContainer = recordFactory
         .newRecordInstance(ContainerLaunchContext.class);
         .newRecordInstance(ContainerLaunchContext.class);
     Resource resource = BuilderUtils.newResource(1024, 1);
     Resource resource = BuilderUtils.newResource(1024, 1);
-    amContainer.setResource(resource);
+    context.setResource(resource);
     amContainer.setApplicationACLs(acls);
     amContainer.setApplicationACLs(acls);
     context.setAMContainerSpec(amContainer);
     context.setAMContainerSpec(amContainer);
     submitRequest.setApplicationSubmissionContext(context);
     submitRequest.setApplicationSubmissionContext(context);

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

@@ -134,7 +134,7 @@ public class TestApplicationMasterLauncher {
     Assert.assertEquals(app.getSubmitTime(),
     Assert.assertEquals(app.getSubmitTime(),
         containerManager.submitTimeAtContainerManager);
         containerManager.submitTimeAtContainerManager);
     Assert.assertEquals(app.getRMAppAttempt(appAttemptId)
     Assert.assertEquals(app.getRMAppAttempt(appAttemptId)
-        .getSubmissionContext().getAMContainerSpec().getContainerId()
+        .getMasterContainer().getId()
         .toString(), containerManager.containerIdAtContainerManager);
         .toString(), containerManager.containerIdAtContainerManager);
     Assert.assertEquals(nm1.getNodeId().getHost(),
     Assert.assertEquals(nm1.getNodeId().getHost(),
         containerManager.nmHostAtContainerManager);
         containerManager.nmHostAtContainerManager);

+ 6 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java

@@ -340,7 +340,7 @@ public class TestClientRMService {
     final SubmitApplicationRequest submitRequest = mockSubmitAppRequest(appId);
     final SubmitApplicationRequest submitRequest = mockSubmitAppRequest(appId);
     Resource resource = Resources.createResource(
     Resource resource = Resources.createResource(
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1);
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MAXIMUM_ALLOCATION_MB + 1);
-    when(submitRequest.getApplicationSubmissionContext().getAMContainerSpec()
+    when(submitRequest.getApplicationSubmissionContext()
         .getResource()).thenReturn(resource);
         .getResource()).thenReturn(resource);
 
 
     final ClientRMService rmService =
     final ClientRMService rmService =
@@ -364,16 +364,17 @@ public class TestClientRMService {
     String queue = MockApps.newQueue();
     String queue = MockApps.newQueue();
 
 
     ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class);
     ContainerLaunchContext amContainerSpec = mock(ContainerLaunchContext.class);
+
     Resource resource = Resources.createResource(
     Resource resource = Resources.createResource(
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
         YarnConfiguration.DEFAULT_RM_SCHEDULER_MINIMUM_ALLOCATION_MB);
-    when(amContainerSpec.getResource()).thenReturn(resource);
 
 
     ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
     ApplicationSubmissionContext submissionContext = mock(ApplicationSubmissionContext.class);
-    when(submissionContext.getUser()).thenReturn(user);
-    when(submissionContext.getQueue()).thenReturn(queue);
     when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
     when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
+    when(submissionContext.getAMContainerSpec().getUser()).thenReturn(user);
+    when(submissionContext.getQueue()).thenReturn(queue);
     when(submissionContext.getApplicationId()).thenReturn(appId);
     when(submissionContext.getApplicationId()).thenReturn(appId);
-    
+    when(submissionContext.getResource()).thenReturn(resource);
+
    SubmitApplicationRequest submitRequest =
    SubmitApplicationRequest submitRequest =
        recordFactory.newRecordInstance(SubmitApplicationRequest.class);
        recordFactory.newRecordInstance(SubmitApplicationRequest.class);
    submitRequest.setApplicationSubmissionContext(submissionContext);
    submitRequest.setApplicationSubmissionContext(submissionContext);

+ 3 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java

@@ -200,14 +200,14 @@ public class TestRMAppAttemptTransitions {
     final String user = MockApps.newUserName();
     final String user = MockApps.newUserName();
     final String queue = MockApps.newQueue();
     final String queue = MockApps.newQueue();
     submissionContext = mock(ApplicationSubmissionContext.class);
     submissionContext = mock(ApplicationSubmissionContext.class);
-    when(submissionContext.getUser()).thenReturn(user);
     when(submissionContext.getQueue()).thenReturn(queue);
     when(submissionContext.getQueue()).thenReturn(queue);
     Resource resource = BuilderUtils.newResource(1536, 1);
     Resource resource = BuilderUtils.newResource(1536, 1);
     ContainerLaunchContext amContainerSpec =
     ContainerLaunchContext amContainerSpec =
-        BuilderUtils.newContainerLaunchContext(null, user, resource, null, null,
+        BuilderUtils.newContainerLaunchContext(user, null, null,
             null, null, null, null);
             null, null, null, null);
     when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
     when(submissionContext.getAMContainerSpec()).thenReturn(amContainerSpec);
-    
+    when(submissionContext.getResource()).thenReturn(resource);
+
     unmanagedAM = false;
     unmanagedAM = false;
     
     
     application = mock(RMApp.class);
     application = mock(RMApp.class);
@@ -494,9 +494,6 @@ public class TestRMAppAttemptTransitions {
     applicationAttempt.handle(
     applicationAttempt.handle(
         new RMAppAttemptStoredEvent(
         new RMAppAttemptStoredEvent(
             applicationAttempt.getAppAttemptId(), null));
             applicationAttempt.getAppAttemptId(), null));
-    assertEquals(resource,
-        applicationAttempt.getSubmissionContext()
-        .getAMContainerSpec().getResource());
     
     
     testAppAttemptAllocatedState(container);
     testAppAttemptAllocatedState(container);
     
     

+ 6 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java

@@ -45,6 +45,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 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.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.Priority;
 import org.apache.hadoop.yarn.api.records.QueueACL;
 import org.apache.hadoop.yarn.api.records.QueueACL;
@@ -72,6 +73,7 @@ import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeAddedSc
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.event.NodeRemovedSchedulerEvent;
 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.fair.modes.FifoSchedulingMode;
 import org.apache.hadoop.yarn.server.resourcemanager.scheduler.fair.modes.FifoSchedulingMode;
+import org.apache.hadoop.yarn.util.BuilderUtils;
 import org.junit.After;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Before;
 import org.junit.Test;
 import org.junit.Test;
@@ -1406,6 +1408,10 @@ public class TestFairScheduler {
     ApplicationMasterService masterService =
     ApplicationMasterService masterService =
         new ApplicationMasterService(resourceManager.getRMContext(), scheduler);
         new ApplicationMasterService(resourceManager.getRMContext(), scheduler);
     ApplicationSubmissionContext submissionContext = new ApplicationSubmissionContextPBImpl();
     ApplicationSubmissionContext submissionContext = new ApplicationSubmissionContextPBImpl();
+    ContainerLaunchContext clc =
+        BuilderUtils.newContainerLaunchContext(user, null, null, null, null,
+            null, null);
+    submissionContext.setAMContainerSpec(clc);
     RMApp application =
     RMApp application =
         new RMAppImpl(applicationId, resourceManager.getRMContext(), conf, name, user, 
         new RMAppImpl(applicationId, resourceManager.getRMContext(), conf, name, user, 
           queue, submissionContext, scheduler, masterService,
           queue, submissionContext, scheduler, masterService,

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/webapp/TestRMWebServicesApps.java

@@ -1079,8 +1079,9 @@ public class TestRMWebServicesApps extends JerseyTest {
         .getMasterContainer().getNodeId().toString(), nodeId);
         .getMasterContainer().getNodeId().toString(), nodeId);
     assertTrue("logsLink doesn't match",
     assertTrue("logsLink doesn't match",
         logsLink.startsWith("http://"));
         logsLink.startsWith("http://"));
-    assertTrue("logsLink doesn't contain user info",
-        logsLink.endsWith("/" + appAttempt.getSubmissionContext().getUser()));
+    assertTrue(
+        "logsLink doesn't contain user info", logsLink.endsWith("/"
+        + appAttempt.getSubmissionContext().getAMContainerSpec().getUser()));
   }
   }
 
 
 }
 }

+ 21 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

@@ -356,8 +356,13 @@ public class TestContainerManagerSecurity {
 
 
         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 =
+            BuilderUtils.newContainer(newTokenId.getContainerID(), null, null,
+                BuilderUtils.newResource(newTokenId.getResource().getMemory(),
+                    newTokenId.getResource().getVirtualCores()), null, null);
         StartContainerRequest request = Records.newRecord(StartContainerRequest.class);
         StartContainerRequest request = Records.newRecord(StartContainerRequest.class);
         request.setContainerLaunchContext(context);
         request.setContainerLaunchContext(context);
+        request.setContainer(container);
 
 
         //Calling startContainer with an expired token.
         //Calling startContainer with an expired token.
         try {
         try {
@@ -397,8 +402,8 @@ public class TestContainerManagerSecurity {
       InterruptedException {
       InterruptedException {
 
 
     ContainerLaunchContext amContainer = BuilderUtils
     ContainerLaunchContext amContainer = BuilderUtils
-        .newContainerLaunchContext(null, "testUser", BuilderUtils
-            .newResource(1024, 1), Collections.<String, LocalResource>emptyMap(),
+        .newContainerLaunchContext("testUser",
+            Collections.<String, LocalResource>emptyMap(),
             new HashMap<String, String>(), Arrays.asList("sleep", "100"),
             new HashMap<String, String>(), Arrays.asList("sleep", "100"),
             new HashMap<String, ByteBuffer>(), null,
             new HashMap<String, ByteBuffer>(), null,
             new HashMap<ApplicationAccessType, String>());
             new HashMap<ApplicationAccessType, String>());
@@ -406,8 +411,9 @@ public class TestContainerManagerSecurity {
     ApplicationSubmissionContext appSubmissionContext = recordFactory
     ApplicationSubmissionContext appSubmissionContext = recordFactory
         .newRecordInstance(ApplicationSubmissionContext.class);
         .newRecordInstance(ApplicationSubmissionContext.class);
     appSubmissionContext.setApplicationId(appID);
     appSubmissionContext.setApplicationId(appID);
-    appSubmissionContext.setUser("testUser");
     appSubmissionContext.setAMContainerSpec(amContainer);
     appSubmissionContext.setAMContainerSpec(amContainer);
+    appSubmissionContext.getAMContainerSpec().setUser("testUser");
+    appSubmissionContext.setResource(BuilderUtils.newResource(1024, 1));
 
 
     SubmitApplicationRequest submitRequest = recordFactory
     SubmitApplicationRequest submitRequest = recordFactory
         .newRecordInstance(SubmitApplicationRequest.class);
         .newRecordInstance(SubmitApplicationRequest.class);
@@ -533,8 +539,11 @@ public class TestContainerManagerSecurity {
     // Authenticated but unauthorized, due to wrong resource
     // Authenticated but unauthorized, due to wrong resource
     ContainerLaunchContext context =
     ContainerLaunchContext context =
         createContainerLaunchContextForTest(tokenId);
         createContainerLaunchContextForTest(tokenId);
-    context.getResource().setMemory(2048); // Set a different resource size.
+    Container container =
+        BuilderUtils.newContainer(tokenId.getContainerID(), null, null,
+            BuilderUtils.newResource(2048, 1), null, null);
     request.setContainerLaunchContext(context);
     request.setContainerLaunchContext(context);
+    request.setContainer(container);
     try {
     try {
       client.startContainer(request);
       client.startContainer(request);
       fail("Connection initiation with unauthorized "
       fail("Connection initiation with unauthorized "
@@ -545,7 +554,7 @@ public class TestContainerManagerSecurity {
           "Unauthorized request to start container. "));
           "Unauthorized request to start container. "));
       Assert.assertTrue(e.getMessage().contains(
       Assert.assertTrue(e.getMessage().contains(
           "\nExpected resource " + tokenId.getResource().toString()
           "\nExpected resource " + tokenId.getResource().toString()
-              + " but found " + context.getResource().toString()));
+              + " but found " + container.getResource().toString()));
     }
     }
   }
   }
 
 
@@ -557,7 +566,12 @@ public class TestContainerManagerSecurity {
     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);
     request.setContainerLaunchContext(context);
     request.setContainerLaunchContext(context);
+    request.setContainer(container);
     try {
     try {
       client.startContainer(request);
       client.startContainer(request);
       fail("Connection initiation with unauthorized "
       fail("Connection initiation with unauthorized "
@@ -575,12 +589,8 @@ public class TestContainerManagerSecurity {
   private ContainerLaunchContext createContainerLaunchContextForTest(
   private ContainerLaunchContext createContainerLaunchContextForTest(
       ContainerTokenIdentifier tokenId) {
       ContainerTokenIdentifier tokenId) {
     ContainerLaunchContext context =
     ContainerLaunchContext context =
-        BuilderUtils.newContainerLaunchContext(tokenId.getContainerID(),
-            "testUser",
-            BuilderUtils.newResource(
-                tokenId.getResource().getMemory(), 
-                tokenId.getResource().getVirtualCores()),
-            new HashMap<String, LocalResource>(),
+        BuilderUtils.newContainerLaunchContext(
+            "testUser", 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>());