浏览代码

YARN-926. Modified ContainerManagerProtcol APIs to take in requests for multiple containers. Contributed by Jian He.
MAPREDUCE-5412. Update MR app to use multiple containers API of ContainerManager after YARN-926. Contributed by Jian He.
svn merge --ignore-ancestry -c 1506391 ../../trunk/


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

Vinod Kumar Vavilapalli 11 年之前
父节点
当前提交
19c496d993
共有 73 个文件被更改,包括 2903 次插入1462 次删除
  1. 3 0
      hadoop-mapreduce-project/CHANGES.txt
  2. 28 16
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/launcher/ContainerLauncherImpl.java
  3. 18 15
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
  4. 27 29
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
  5. 3 0
      hadoop-yarn-project/CHANGES.txt
  6. 5 0
      hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml
  7. 80 53
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java
  8. 0 61
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java
  9. 28 19
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java
  10. 91 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java
  11. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerRequest.java
  12. 76 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainersRequest.java
  13. 56 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainersResponse.java
  14. 0 47
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java
  15. 15 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainersRequest.java
  16. 90 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainersResponse.java
  17. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java
  18. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerLaunchContext.java
  19. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java
  20. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java
  21. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java
  22. 93 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SerializedException.java
  23. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/InvalidContainerException.java
  24. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/NMNotYetReadyException.java
  25. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationInitializationContext.java
  26. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/AuxiliaryService.java
  27. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto
  28. 34 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/yarn_service_protos.proto
  29. 59 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java
  30. 32 32
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java
  31. 31 31
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java
  32. 0 130
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusResponsePBImpl.java
  33. 58 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesRequestPBImpl.java
  34. 223 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesResponsePBImpl.java
  35. 0 176
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerResponsePBImpl.java
  36. 139 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersRequestPBImpl.java
  37. 319 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersResponsePBImpl.java
  38. 0 130
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainerRequestPBImpl.java
  39. 0 68
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainerResponsePBImpl.java
  40. 146 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersRequestPBImpl.java
  41. 234 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersResponsePBImpl.java
  42. 59 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java
  43. 31 23
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
  44. 61 46
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  45. 0 39
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/SerializedException.java
  46. 1 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java
  47. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalResourceStatus.java
  48. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalResourceStatusPBImpl.java
  49. 121 78
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/ContainerManagerImpl.java
  50. 1 1
      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
  51. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java
  52. 2 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java
  53. 24 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java
  54. 20 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
  55. 18 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
  56. 11 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  57. 17 13
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
  58. 1 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  59. 35 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java
  60. 19 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java
  61. 237 69
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  62. 39 25
      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
  63. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java
  64. 3 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java
  65. 12 7
      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
  66. 15 10
      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
  67. 49 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/main/java/org/apache/hadoop/yarn/server/resourcemanager/amlauncher/AMLauncher.java
  68. 16 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java
  69. 113 110
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java
  70. 15 15
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java
  71. 19 12
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationMasterLauncher.java
  72. 7 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java
  73. 39 14
      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

@@ -414,6 +414,9 @@ Release 2.1.0-beta - 2013-07-02
     MAPREDUCE-5325. MR changes related to YARN-727. ClientRMProtocol.getAllApplications
     should accept ApplicationType as a parameter. (Xuan Gong via hitesh)
 
+    MAPREDUCE-5412. Update MR app to use multiple containers API of
+    ContainerManager after YARN-926. (Jian He via vinodkv)
+
   BREAKDOWN OF HADOOP-8562 SUBTASKS
 
     MAPREDUCE-4739. Some MapReduce tests fail to find winutils.

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

@@ -20,7 +20,9 @@ package org.apache.hadoop.mapreduce.v2.app.launcher;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Set;
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.ConcurrentHashMap;
@@ -44,14 +46,15 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy;
 import org.apache.hadoop.yarn.client.api.impl.ContainerManagementProtocolProxy.ContainerManagementProtocolProxyData;
 import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
-import org.apache.hadoop.yarn.util.Records;
 
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
@@ -139,13 +142,18 @@ public class ContainerLauncherImpl extends AbstractService implements
           event.getContainerLaunchContext();
 
         // Now launch the actual container
-        StartContainerRequest startRequest = Records
-          .newRecord(StartContainerRequest.class);
-        startRequest.setContainerLaunchContext(containerLaunchContext);
-        startRequest.setContainerToken(event.getContainerToken());
-        StartContainerResponse response =
-            proxy.getContainerManagementProtocol().startContainer(startRequest);
-
+        StartContainerRequest startRequest =
+            StartContainerRequest.newInstance(containerLaunchContext,
+              event.getContainerToken());
+        List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+        list.add(startRequest);
+        StartContainersRequest requestList = StartContainersRequest.newInstance(list);
+        StartContainersResponse response =
+            proxy.getContainerManagementProtocol().startContainers(requestList);
+        if (response.getFailedRequests() != null
+            && response.getFailedRequests().containsKey(containerID)) {
+          throw response.getFailedRequests().get(containerID).deSerialize();
+        }
         ByteBuffer portInfo =
             response.getAllServicesMetaData().get(
                 ShuffleHandler.MAPREDUCE_SHUFFLE_SERVICEID);
@@ -192,13 +200,17 @@ public class ContainerLauncherImpl extends AbstractService implements
           proxy = getCMProxy(this.containerMgrAddress, this.containerID);
 
           // kill the remote container if already launched
-          StopContainerRequest stopRequest = Records
-              .newRecord(StopContainerRequest.class);
-          stopRequest.setContainerId(this.containerID);
-          proxy.getContainerManagementProtocol().stopContainer(stopRequest);
-
+          List<ContainerId> ids = new ArrayList<ContainerId>();
+          ids.add(this.containerID);
+          StopContainersRequest request = StopContainersRequest.newInstance(ids);
+          StopContainersResponse response =
+              proxy.getContainerManagementProtocol().stopContainers(request);
+          if (response.getFailedRequests() != null
+              && response.getFailedRequests().containsKey(this.containerID)) {
+            throw response.getFailedRequests().get(this.containerID)
+              .deSerialize();
+          }
         } catch (Throwable t) {
-
           // ignore the cleanup failure
           String message = "cleanup failed for container "
               + this.containerID + " : "

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

@@ -24,6 +24,8 @@ import java.io.IOException;
 import java.lang.reflect.UndeclaredThrowableException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.atomic.AtomicInteger;
@@ -52,12 +54,13 @@ import org.apache.hadoop.mapreduce.v2.app.job.impl.TaskAttemptImpl;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -393,18 +396,18 @@ public class TestContainerLauncher {
     private ContainerStatus status = null;
 
     @Override
-    public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request) throws IOException {
-      GetContainerStatusResponse response = recordFactory
-          .newRecordInstance(GetContainerStatusResponse.class);
-      response.setStatus(status);
-      return response;
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request) throws IOException {
+      List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
+      statuses.add(status);
+      return GetContainerStatusesResponse.newInstance(statuses, null);
     }
 
     @Override
-    public StartContainerResponse startContainer(StartContainerRequest request)
+    public StartContainersResponse startContainers(StartContainersRequest requests)
         throws IOException {
 
+      StartContainerRequest request = requests.getStartContainerRequests().get(0);
       ContainerTokenIdentifier containerTokenIdentifier =
           MRApp.newContainerTokenIdentifier(request.getContainerToken());
 
@@ -412,8 +415,8 @@ public class TestContainerLauncher {
       Assert.assertEquals(MRApp.NM_HOST + ":" + MRApp.NM_PORT,
         containerTokenIdentifier.getNmHostAddress());
 
-      StartContainerResponse response = recordFactory
-          .newRecordInstance(StartContainerResponse.class);
+      StartContainersResponse response = recordFactory
+          .newRecordInstance(StartContainersResponse.class);
       status = recordFactory.newRecordInstance(ContainerStatus.class);
       try {
         // make the thread sleep to look like its not going to respond
@@ -429,7 +432,7 @@ public class TestContainerLauncher {
     }
 
     @Override
-    public StopContainerResponse stopContainer(StopContainerRequest request)
+    public StopContainersResponse stopContainers(StopContainersRequest request)
         throws IOException {
       Exception e = new Exception("Dummy function", new Exception(
           "Dummy function cause"));

+ 27 - 29
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java

@@ -45,12 +45,12 @@ import org.apache.hadoop.mapreduce.v2.app.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app.launcher.ContainerLauncher.EventType;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -162,8 +162,8 @@ public class TestContainerLauncherImpl {
     try {
       ContainerId contId = makeContainerId(0l, 0, 0, 1);
       TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
-      StartContainerResponse startResp = 
-        recordFactory.newRecordInstance(StartContainerResponse.class);
+      StartContainersResponse startResp =
+        recordFactory.newRecordInstance(StartContainersResponse.class);
       startResp.setAllServicesMetaData(serviceResponse);
       
 
@@ -176,14 +176,14 @@ public class TestContainerLauncherImpl {
         .thenReturn(contId);
       when(mockLaunchEvent.getTaskAttemptID()).thenReturn(taskAttemptId);
       when(mockLaunchEvent.getContainerMgrAddress()).thenReturn(cmAddress);
-      when(mockCM.startContainer(any(StartContainerRequest.class))).thenReturn(startResp);
+      when(mockCM.startContainers(any(StartContainersRequest.class))).thenReturn(startResp);
       when(mockLaunchEvent.getContainerToken()).thenReturn(
           createNewContainerToken(contId, cmAddress));
       ut.handle(mockLaunchEvent);
       
       ut.waitForPoolToIdle();
       
-      verify(mockCM).startContainer(any(StartContainerRequest.class));
+      verify(mockCM).startContainers(any(StartContainersRequest.class));
       
       LOG.info("inserting cleanup event");
       ContainerLauncherEvent mockCleanupEvent = 
@@ -198,7 +198,7 @@ public class TestContainerLauncherImpl {
       
       ut.waitForPoolToIdle();
       
-      verify(mockCM).stopContainer(any(StopContainerRequest.class));
+      verify(mockCM).stopContainers(any(StopContainersRequest.class));
     } finally {
       ut.stop();
     }
@@ -224,8 +224,8 @@ public class TestContainerLauncherImpl {
       ContainerId contId = makeContainerId(0l, 0, 0, 1);
       TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
       String cmAddress = "127.0.0.1:8000";
-      StartContainerResponse startResp = 
-        recordFactory.newRecordInstance(StartContainerResponse.class);
+      StartContainersResponse startResp =
+        recordFactory.newRecordInstance(StartContainersResponse.class);
       startResp.setAllServicesMetaData(serviceResponse);
 
       LOG.info("inserting cleanup event");
@@ -241,7 +241,7 @@ public class TestContainerLauncherImpl {
       
       ut.waitForPoolToIdle();
       
-      verify(mockCM, never()).stopContainer(any(StopContainerRequest.class));
+      verify(mockCM, never()).stopContainers(any(StopContainersRequest.class));
 
       LOG.info("inserting launch event");
       ContainerRemoteLaunchEvent mockLaunchEvent = 
@@ -252,14 +252,14 @@ public class TestContainerLauncherImpl {
         .thenReturn(contId);
       when(mockLaunchEvent.getTaskAttemptID()).thenReturn(taskAttemptId);
       when(mockLaunchEvent.getContainerMgrAddress()).thenReturn(cmAddress);
-      when(mockCM.startContainer(any(StartContainerRequest.class))).thenReturn(startResp);
+      when(mockCM.startContainers(any(StartContainersRequest.class))).thenReturn(startResp);
       when(mockLaunchEvent.getContainerToken()).thenReturn(
           createNewContainerToken(contId, cmAddress));
       ut.handle(mockLaunchEvent);
       
       ut.waitForPoolToIdle();
       
-      verify(mockCM, never()).startContainer(any(StartContainerRequest.class));
+      verify(mockCM, never()).startContainers(any(StartContainersRequest.class));
     } finally {
       ut.stop();
     }
@@ -286,8 +286,8 @@ public class TestContainerLauncherImpl {
       ContainerId contId = makeContainerId(0l, 0, 0, 1);
       TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
       String cmAddress = "127.0.0.1:8000";
-      StartContainerResponse startResp =
-        recordFactory.newRecordInstance(StartContainerResponse.class);
+      StartContainersResponse startResp =
+        recordFactory.newRecordInstance(StartContainersResponse.class);
       startResp.setAllServicesMetaData(serviceResponse);
 
       LOG.info("inserting launch event");
@@ -299,20 +299,20 @@ public class TestContainerLauncherImpl {
         .thenReturn(contId);
       when(mockLaunchEvent.getTaskAttemptID()).thenReturn(taskAttemptId);
       when(mockLaunchEvent.getContainerMgrAddress()).thenReturn(cmAddress);
-      when(mockCM.startContainer(any(StartContainerRequest.class))).thenReturn(startResp);
+      when(mockCM.startContainers(any(StartContainersRequest.class))).thenReturn(startResp);
       when(mockLaunchEvent.getContainerToken()).thenReturn(
           createNewContainerToken(contId, cmAddress));
       ut.handle(mockLaunchEvent);
 
       ut.waitForPoolToIdle();
 
-      verify(mockCM).startContainer(any(StartContainerRequest.class));
+      verify(mockCM).startContainers(any(StartContainersRequest.class));
 
       // skip cleanup and make sure stop kills the container
 
     } finally {
       ut.stop();
-      verify(mockCM).stopContainer(any(StopContainerRequest.class));
+      verify(mockCM).stopContainers(any(StopContainersRequest.class));
     }
   }
   
@@ -341,8 +341,8 @@ public class TestContainerLauncherImpl {
       ContainerId contId = makeContainerId(0l, 0, 0, 1);
       TaskAttemptId taskAttemptId = makeTaskAttemptId(0l, 0, 0, TaskType.MAP, 0);
       String cmAddress = "127.0.0.1:8000";
-      StartContainerResponse startResp = 
-        recordFactory.newRecordInstance(StartContainerResponse.class);
+      StartContainersResponse startResp =
+        recordFactory.newRecordInstance(StartContainersResponse.class);
       startResp.setAllServicesMetaData(serviceResponse);
       
      
@@ -415,7 +415,7 @@ public class TestContainerLauncherImpl {
       this.completeLaunchBarrier = completeLaunchBarrier;
     }
     @Override
-    public StartContainerResponse startContainer(StartContainerRequest request)
+    public StartContainersResponse startContainers(StartContainersRequest request)
         throws IOException {
       try {
         startLaunchBarrier.await();
@@ -433,16 +433,14 @@ public class TestContainerLauncherImpl {
     }
 
     @Override
-    public StopContainerResponse stopContainer(StopContainerRequest request)
+    public StopContainersResponse stopContainers(StopContainersRequest request)
         throws IOException {
-    
       return null;
     }
 
     @Override
-    public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request) throws IOException {
-    
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request) throws IOException {
       return null;
     }
   }

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

@@ -194,6 +194,9 @@ Release 2.1.0-beta - 2013-07-02
     YARN-918. Remove ApplicationAttemptId from
     RegisterApplicationMasterRequestProto. (vinodkv via acmurthy)
 
+    YARN-926. Modified ContainerManagerProtcol APIs to take in requests for
+    multiple containers. (Jian He via vinodkv)
+
   NEW FEATURES
 
     YARN-482. FS: Extend SchedulingMode to intermediate queues. 

+ 5 - 0
hadoop-yarn-project/hadoop-yarn/dev-support/findbugs-exclude.xml

@@ -131,6 +131,11 @@
     <Class name="org.apache.hadoop.yarn.server.resourcemanager.resource.Priority$Comparator" />
     <Bug pattern="SE_COMPARATOR_SHOULD_BE_SERIALIZABLE" />
   </Match>
+  <!-- Ignore some irrelevant class name warning -->
+  <Match>
+    <Class name="org.apache.hadoop.yarn.api.records.SerializedException" />
+    <Bug pattern="NM_CLASS_NOT_EXCEPTION" />
+  </Match>
 
   <!-- Inconsistent sync warning - only start() is synchronized-->
   <Match>

+ 80 - 53
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/ContainerManagementProtocol.java

@@ -22,17 +22,17 @@ import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
 import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 
@@ -50,10 +50,12 @@ import org.apache.hadoop.yarn.exceptions.YarnException;
 @Public
 @Stable
 public interface ContainerManagementProtocol {
+
   /**
    * <p>
-   * The <code>ApplicationMaster</code> requests a <code>NodeManager</code> to
-   * <em>start</em> a {@link Container} allocated to it using this interface.
+   * The <code>ApplicationMaster</code> provides a list of
+   * {@link StartContainerRequest}s to a <code>NodeManager</code> to
+   * <em>start</em> {@link Container}s allocated to it using this interface.
    * </p>
    * 
    * <p>
@@ -65,82 +67,107 @@ public interface ContainerManagementProtocol {
    * </p>
    * 
    * <p>
-   * Currently the <code>NodeManager</code> sends an immediate, empty response
-   * via {@link StartContainerResponse} to signify acceptance of the request and
-   * throws an exception in case of errors. The <code>ApplicationMaster</code>
-   * can use {@link #getContainerStatus(GetContainerStatusRequest)} to get
-   * updated status of the to-be-launched or launched container.
+   * The <code>NodeManager</code> sends a response via
+   * {@link StartContainersResponse} which includes a list of
+   * {@link Container}s of successfully launched {@link Container}s, a
+   * containerId-to-exception map for each failed {@link StartContainerRequest} in
+   * which the exception indicates errors from per container and a
+   * allServicesMetaData map between the names of auxiliary services and their
+   * corresponding meta-data. Note: None-container-specific exceptions will
+   * still be thrown by the API method itself.
+   * </p>
+   * <p>
+   * The <code>ApplicationMaster</code> can use
+   * {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
+   * statuses of the to-be-launched or launched containers.
    * </p>
    * 
    * @param request
-   *          request to start a container
-   * @return empty response to indicate acceptance of the request or an
-   *         exception
+   *          request to start a list of containers
+   * @return response including conatinerIds of all successfully launched
+   *         containers, a containerId-to-exception map for failed requests and
+   *         a allServicesMetaData map.
    * @throws YarnException
    * @throws IOException
    * @throws NMNotYetReadyException
    *           This exception is thrown when NM starts from scratch but has not
    *           yet connected with RM.
-   * @throws InvalidContainerException
-   *           This exception is thrown when NM is rejecting start-container
-   *           requests for containers allocated by a previous instance of the
-   *           RM
    */
   @Public
   @Stable
-  StartContainerResponse startContainer(StartContainerRequest request)
+  StartContainersResponse startContainers(StartContainersRequest request)
       throws YarnException, IOException;
 
   /**
-   * <p>The <code>ApplicationMaster</code> requests a <code>NodeManager</code>
-   * to <em>stop</em> a {@link Container} allocated to it using this interface.
+   * <p>
+   * The <code>ApplicationMaster</code> requests a <code>NodeManager</code> to
+   * <em>stop</em> a list of {@link Container}s allocated to it using this
+   * interface.
    * </p>
    * 
-   * <p>The <code>ApplicationMaster</code> sends a
-   * {@link StopContainerRequest} which includes the {@link ContainerId} of the
-   * container to be stopped.</p>
+   * <p>
+   * The <code>ApplicationMaster</code> sends a {@link StopContainersRequest}
+   * which includes the {@link ContainerId}s of the containers to be stopped.
+   * </p>
    * 
-   * <p>Currently the <code>NodeManager</code> sends an immediate, empty 
-   * response via {@link StopContainerResponse} to signify acceptance of the
-   * request and throws an exception in case of errors. The 
-   * <code>ApplicationMaster</code> can use 
-   * {@link #getContainerStatus(GetContainerStatusRequest)} to get updated 
-   * status of the container.</p>
+   * <p>
+   * The <code>NodeManager</code> sends a response via
+   * {@link StopContainersResponse} which includes a list of {@link ContainerId}
+   * s of successfully stopped containers, a containerId-to-exception map for
+   * each failed request in which the exception indicates errors from per
+   * container. Note: None-container-specific exceptions will still be thrown by
+   * the API method itself. <code>ApplicationMaster</code> can use
+   * {@link #getContainerStatuses(GetContainerStatusesRequest)} to get updated
+   * statuses of the containers.
+   * </p>
    * 
-   * @param request request to stop a container
-   * @return empty response to indicate acceptance of the request 
-   *         or an exception
+   * @param request
+   *          request to stop a list of containers
+   * @return response which includes a list of containerIds of successfully
+   *         stopped containers, a containerId-to-exception map for failed
+   *         requests.
    * @throws YarnException
    * @throws IOException
    */
   @Public
   @Stable
-  StopContainerResponse stopContainer(StopContainerRequest request)
+  StopContainersResponse stopContainers(StopContainersRequest request)
       throws YarnException, IOException;
 
   /**
-   * <p>The api used by the <code>ApplicationMaster</code> to request for 
-   * current status of a <code>Container</code> from the 
-   * <code>NodeManager</code>.</p>
+   * <p>
+   * The API used by the <code>ApplicationMaster</code> to request for current
+   * statuses of <code>Container</code>s from the <code>NodeManager</code>.
+   * </p>
+   * 
+   * <p>
+   * The <code>ApplicationMaster</code> sends a
+   * {@link GetContainerStatusesRequest} which includes the {@link ContainerId}s
+   * of all containers whose statuses are needed.
+   * </p>
+   * 
+   * <p>
+   * The <code>NodeManager</code> responds with
+   * {@link GetContainerStatusesResponse} which includes a list of
+   * {@link ContainerStatus} of the successfully queried containers and a
+   * containerId-to-exception map for each failed request in which the exception
+   * indicates errors from per container. Note: None-container-specific
+   * exceptions will still be thrown by the API method itself.
+   * </p>
+   * 
+   * @param request
+   *          request to get <code>ContainerStatus</code>es of containers with
+   *          the specified <code>ContainerId</code>s
+   * @return response containing the list of <code>ContainerStatus</code> of the
+   *         successfully queried containers and a containerId-to-exception map
+   *         for failed requests.
    * 
-   * <p>The <code>ApplicationMaster</code> sends a
-   * {@link GetContainerStatusRequest} which includes the {@link ContainerId} of
-   * the container whose status is needed.</p>
-   *
-   *<p>The <code>NodeManager</code> responds with 
-   *{@link GetContainerStatusResponse} which includes the 
-   *{@link ContainerStatus} of the container.</p>
-   *
-   * @param request request to get <code>ContainerStatus</code> of a container
-   *                with the specified <code>ContainerId</code>
-   * @return response containing the <code>ContainerStatus</code> of the
-   * container
    * @throws YarnException
    * @throws IOException
    */
   @Public
   @Stable
-  GetContainerStatusResponse getContainerStatus(
-      GetContainerStatusRequest request) throws YarnException,
+  GetContainerStatusesResponse getContainerStatuses(
+      GetContainerStatusesRequest request) throws YarnException,
       IOException;
 }

+ 0 - 61
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusResponse.java

@@ -1,61 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.protocolrecords;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * <p>The response sent by the <code>NodeManager</code> to the 
- * <code>ApplicationMaster</code> when asked to obtain the <em>status</em> 
- * of a container.</p>
- * 
- * @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest)
- */
-@Public
-@Stable
-public abstract class GetContainerStatusResponse {
-
-  @Private
-  @Unstable
-  public static GetContainerStatusResponse newInstance(
-      ContainerStatus containerStatus) {
-    GetContainerStatusResponse response =
-        Records.newRecord(GetContainerStatusResponse.class);
-    response.setStatus(containerStatus);
-    return response;
-  }
-
-  /**
-   * Get the <code>ContainerStatus</code> of the container.
-   * @return <code>ContainerStatus</code> of the container
-   */
-  @Public
-  @Stable
-  public abstract ContainerStatus getStatus();
-  
-  @Private
-  @Unstable
-  public abstract void setStatus(ContainerStatus containerStatus);
-}

+ 28 - 19
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusRequest.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesRequest.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
@@ -26,41 +28,48 @@ import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The request sent by the <code>ApplicationMaster</code> to the
- * <code>NodeManager</code> to get {@link ContainerStatus} of a container.</p>
+ * <p>
+ * The request sent by the <code>ApplicationMaster</code> to the
+ * <code>NodeManager</code> to get {@link ContainerStatus} of requested
+ * containers.
+ * </p>
  * 
- * @see ContainerManagementProtocol#getContainerStatus(GetContainerStatusRequest)
+ * @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
  */
 @Public
 @Stable
-public abstract class GetContainerStatusRequest {
+public abstract class GetContainerStatusesRequest {
 
   @Public
   @Stable
-  public static GetContainerStatusRequest newInstance(ContainerId containerId) {
-    GetContainerStatusRequest request =
-        Records.newRecord(GetContainerStatusRequest.class);
-    request.setContainerId(containerId);
+  public static GetContainerStatusesRequest newInstance(
+      List<ContainerId> containerIds) {
+    GetContainerStatusesRequest request =
+        Records.newRecord(GetContainerStatusesRequest.class);
+    request.setContainerIds(containerIds);
     return request;
   }
 
   /**
-   * Get the <code>ContainerId</code> of container for which to obtain the
-   * <code>ContainerStatus</code>.
-   * @return <code>ContainerId</code> of container for which to obtain the
-   *         <code>ContainerStatus</code>
+   * Get the list of <code>ContainerId</code>s of containers for which to obtain
+   * the <code>ContainerStatus</code>.
+   * 
+   * @return the list of <code>ContainerId</code>s of containers for which to
+   *         obtain the <code>ContainerStatus</code>.
    */
   @Public
   @Stable
-  public abstract ContainerId getContainerId();
-  
+  public abstract List<ContainerId> getContainerIds();
+
   /**
-   * Set the <code>ContainerId</code> of container for which to obtain the
-   * <code>ContainerStatus</code>
-   * @param containerId <code>ContainerId</code> of container for which to 
-   *                    obtain the <code>ContainerStatus</code>
+   * Set a list of <code>ContainerId</code>s of containers for which to obtain
+   * the <code>ContainerStatus</code>
+   * 
+   * @param containerIds
+   *          a list of <code>ContainerId</code>s of containers for which to
+   *          obtain the <code>ContainerStatus</code>
    */
   @Public
   @Stable
-  public abstract void setContainerId(ContainerId containerId);
+  public abstract void setContainerIds(List<ContainerId> containerIds);
 }

+ 91 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/GetContainerStatusesResponse.java

@@ -0,0 +1,91 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The response sent by the <code>NodeManager</code> to the
+ * <code>ApplicationMaster</code> when asked to obtain the
+ * <code>ContainerStatus</code> of requested containers.
+ * </p>
+ * 
+ * @see ContainerManagementProtocol#getContainerStatuses(GetContainerStatusesRequest)
+ */
+@Public
+@Stable
+public abstract class GetContainerStatusesResponse {
+
+  @Private
+  @Unstable
+  public static GetContainerStatusesResponse newInstance(
+      List<ContainerStatus> statuses,
+      Map<ContainerId, SerializedException> failedRequests) {
+    GetContainerStatusesResponse response =
+        Records.newRecord(GetContainerStatusesResponse.class);
+    response.setContainerStatuses(statuses);
+    response.setFailedRequests(failedRequests);
+    return response;
+  }
+
+  /**
+   * Get the <code>ContainerStatus</code>es of the requested containers.
+   * 
+   * @return <code>ContainerStatus</code>es of the requested containers.
+   */
+  @Public
+  @Stable
+  public abstract List<ContainerStatus> getContainerStatuses();
+
+  /**
+   * Set the <code>ContainerStatus</code>es of the requested containers.
+   */
+  @Private
+  @Unstable
+  public abstract void setContainerStatuses(List<ContainerStatus> statuses);
+
+  /**
+   * Get the containerId-to-exception map in which the exception indicates error
+   * from per container for failed requests
+   */
+  @Public
+  @Stable
+  public abstract Map<ContainerId, SerializedException> getFailedRequests();
+
+  /**
+   * Set the containerId-to-exception map in which the exception indicates error
+   * from per container for failed requests
+   */
+  @Private
+  @Unstable
+  public abstract void setFailedRequests(
+      Map<ContainerId, SerializedException> failedContainers);
+}

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

@@ -36,7 +36,7 @@ import org.apache.hadoop.yarn.util.Records;
  * necessary binaries/jar/shared-objects etc. via the 
  * {@link ContainerLaunchContext}.</p>
  *
- * @see ContainerManagementProtocol#startContainer(StartContainerRequest)
+ * @see ContainerManagementProtocol#startContainers(StartContainersRequest)
  */
 @Public
 @Stable
@@ -81,7 +81,7 @@ public abstract class StartContainerRequest {
    * @return the container token to be used for authorization during starting
    * container.
    * @see NMToken
-   * @see ContainerManagementProtocol#startContainer(StartContainerRequest)
+   * @see ContainerManagementProtocol#startContainers(StartContainersRequest)
    */
   @Public
   @Stable

+ 76 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainersRequest.java

@@ -0,0 +1,76 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords;
+
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The request which contains a list of {@link StartContainerRequest} sent by
+ * the <code>ApplicationMaster</code> to the <code>NodeManager</code> to
+ * <em>start</em> containers.
+ * </p>
+ * 
+ * <p>
+ * In each {@link StartContainerRequest}, the <code>ApplicationMaster</code> has
+ * to provide details such as allocated resource capability, security tokens (if
+ * enabled), command to be executed to start the container, environment for the
+ * process, necessary binaries/jar/shared-objects etc. via the
+ * {@link ContainerLaunchContext}.
+ * </p>
+ * 
+ * @see ContainerManagementProtocol#startContainers(StartContainersRequest)
+ */
+@Public
+@Stable
+public abstract class StartContainersRequest {
+
+  @Public
+  @Stable
+  public static StartContainersRequest newInstance(
+      List<StartContainerRequest> requests) {
+    StartContainersRequest request =
+        Records.newRecord(StartContainersRequest.class);
+    request.setStartContainerRequests(requests);
+    return request;
+  }
+
+  /**
+   * Get a list of {@link StartContainerRequest} to start containers.
+   * @return a list of {@link StartContainerRequest} to start containers.
+   */
+  @Public
+  @Stable
+  public abstract List<StartContainerRequest> getStartContainerRequests();
+
+  /**
+   * Set a list of {@link StartContainerRequest} to start containers.
+   * @param request a list of {@link StartContainerRequest} to start containers
+   */
+  @Public
+  @Stable
+  public abstract void setStartContainerRequests(
+      List<StartContainerRequest> request);
+}

+ 56 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainerResponse.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StartContainersResponse.java

@@ -19,6 +19,7 @@
 package org.apache.hadoop.yarn.api.protocolrecords;
 
 import java.nio.ByteBuffer;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -26,29 +27,71 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
- * <p>The response sent by the <code>NodeManager</code> to the 
- * <code>ApplicationMaster</code> when asked to <em>start</em> an
- * allocated container.</p>
+ * <p>
+ * The response sent by the <code>NodeManager</code> to the
+ * <code>ApplicationMaster</code> when asked to <em>start</em> an allocated
+ * container.
+ * </p>
  * 
- * @see ContainerManagementProtocol#startContainer(StartContainerRequest)
+ * @see ContainerManagementProtocol#startContainers(StartContainersRequest)
  */
 @Public
 @Stable
-public abstract class StartContainerResponse {
+public abstract class StartContainersResponse {
 
   @Private
   @Unstable
-  public static StartContainerResponse newInstance(
-      Map<String, ByteBuffer> servicesMetaData) {
-    StartContainerResponse response =
-        Records.newRecord(StartContainerResponse.class);
+  public static StartContainersResponse newInstance(
+      Map<String, ByteBuffer> servicesMetaData,
+      List<ContainerId> succeededContainers,
+      Map<ContainerId, SerializedException> failedContainers) {
+    StartContainersResponse response =
+        Records.newRecord(StartContainersResponse.class);
     response.setAllServicesMetaData(servicesMetaData);
+    response.setSuccessfullyStartedContainers(succeededContainers);
+    response.setFailedRequests(failedContainers);
     return response;
   }
 
+  /**
+   * Get the list of <code>ContainerId</code> s of the containers that are
+   * started successfully.
+   * 
+   * @return the list of <code>ContainerId</code> s of the containers that are
+   *         started successfully.
+   * @see ContainerManagementProtocol#startContainers(StartContainersRequest)
+   */
+  @Public
+  @Stable
+  public abstract List<ContainerId> getSuccessfullyStartedContainers();
+
+  @Private
+  @Unstable
+  public abstract void setSuccessfullyStartedContainers(
+      List<ContainerId> succeededContainers);
+
+  /**
+   * Get the containerId-to-exception map in which the exception indicates error
+   * from per container for failed requests
+   */
+  @Public
+  @Stable
+  public abstract Map<ContainerId, SerializedException> getFailedRequests();
+
+  /**
+   * Set the containerId-to-exception map in which the exception indicates error
+   * from per container for failed requests
+   */
+  @Private
+  @Unstable
+  public abstract void setFailedRequests(
+      Map<ContainerId, SerializedException> failedContainers);
+
   /**
    * <p>
    * Get the meta-data from all auxiliary services running on the
@@ -76,8 +119,10 @@ public abstract class StartContainerResponse {
    * Set to the list of auxiliary services which have been started on the
    * <code>NodeManager</code>. This is done only once when the
    * <code>NodeManager</code> starts up
-   * @param allServicesMetaData A map from auxiliary service names to the opaque
-   * blob <code>ByteBuffer</code> for that auxiliary service
+   * 
+   * @param allServicesMetaData
+   *          A map from auxiliary service names to the opaque blob
+   *          <code>ByteBuffer</code> for that auxiliary service
    */
   @Private
   @Unstable

+ 0 - 47
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerResponse.java

@@ -1,47 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.api.protocolrecords;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceAudience.Public;
-import org.apache.hadoop.classification.InterfaceStability.Stable;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.util.Records;
-
-/**
- * <p>The response sent by the <code>NodeManager</code> to the 
- * <code>ApplicationMaster</code> when asked to <em>stop</em> an
- * allocated container.</p>
- * 
- * <p>Currently, this is empty.</p>
- * 
- * @see ContainerManagementProtocol#stopContainer(StopContainerRequest)
- */
-@Public
-@Stable
-public abstract class StopContainerResponse {
-  @Private
-  @Unstable
-  public static StopContainerResponse newInstance() {
-    StopContainerResponse response =
-        Records.newRecord(StopContainerResponse.class);
-    return response;
-  }
-}

+ 15 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainerRequest.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainersRequest.java

@@ -18,6 +18,8 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords;
 
+import java.util.List;
+
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Stable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
@@ -26,36 +28,36 @@ import org.apache.hadoop.yarn.util.Records;
 
 /**
  * <p>The request sent by the <code>ApplicationMaster</code> to the
- * <code>NodeManager</code> to <em>stop</em> a container.</p>
+ * <code>NodeManager</code> to <em>stop</em> containers.</p>
  * 
- * @see ContainerManagementProtocol#stopContainer(StopContainerRequest)
+ * @see ContainerManagementProtocol#stopContainers(StopContainersRequest)
  */
 @Public
 @Stable
-public abstract class StopContainerRequest {
+public abstract class StopContainersRequest {
 
   @Public
   @Stable
-  public static StopContainerRequest newInstance(ContainerId containerId) {
-    StopContainerRequest request =
-        Records.newRecord(StopContainerRequest.class);
-    request.setContainerId(containerId);
+  public static StopContainersRequest newInstance(List<ContainerId> containerIds) {
+    StopContainersRequest request =
+        Records.newRecord(StopContainersRequest.class);
+    request.setContainerIds(containerIds);
     return request;
   }
 
   /**
-   * Get the <code>ContainerId</code> of the container to be stopped.
-   * @return <code>ContainerId</code> of container to be stopped
+   * Get the <code>ContainerId</code>s of the containers to be stopped.
+   * @return <code>ContainerId</code>s of containers to be stopped
    */
   @Public
   @Stable
-  public abstract ContainerId getContainerId();
+  public abstract List<ContainerId> getContainerIds();
   
   /**
-   * Set the <code>ContainerId</code> of the container to be stopped.
-   * @param containerId <code>ContainerId</code> of the container to be stopped
+   * Set the <code>ContainerId</code>s of the containers to be stopped.
+   * @param containerIds <code>ContainerId</code>s of the containers to be stopped
    */
   @Public
   @Stable
-  public abstract void setContainerId(ContainerId containerId);
+  public abstract void setContainerIds(List<ContainerId> containerIds);
 }

+ 90 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/StopContainersResponse.java

@@ -0,0 +1,90 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords;
+
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceAudience.Public;
+import org.apache.hadoop.classification.InterfaceStability.Stable;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.util.Records;
+
+/**
+ * <p>
+ * The response sent by the <code>NodeManager</code> to the
+ * <code>ApplicationMaster</code> when asked to <em>stop</em> allocated
+ * containers.
+ * </p>
+ * 
+ * @see ContainerManagementProtocol#stopContainers(StopContainersRequest)
+ */
+@Public
+@Stable
+public abstract class StopContainersResponse {
+  @Private
+  @Unstable
+  public static StopContainersResponse newInstance(
+      List<ContainerId> succeededRequests,
+      Map<ContainerId, SerializedException> failedRequests) {
+    StopContainersResponse response =
+        Records.newRecord(StopContainersResponse.class);
+    response.setFailedRequests(failedRequests);
+    response.setSuccessfullyStoppedContainers(succeededRequests);
+    return response;
+  }
+
+  /**
+   * Get the list of containerIds of successfully stopped containers.
+   * 
+   * @return the list of containerIds of successfully stopped containers.
+   */
+  @Public
+  @Stable
+  public abstract List<ContainerId> getSuccessfullyStoppedContainers();
+
+  /**
+   * Set the list of containerIds of successfully stopped containers.
+   */
+  @Private
+  @Unstable
+  public abstract void setSuccessfullyStoppedContainers(
+      List<ContainerId> succeededRequests);
+
+  /**
+   * Get the containerId-to-exception map in which the exception indicates error
+   * from per container for failed requests
+   */
+  @Public
+  @Stable
+  public abstract Map<ContainerId, SerializedException> getFailedRequests();
+
+  /**
+   * Set the containerId-to-exception map in which the exception indicates error
+   * from per container for failed requests
+   */
+  @Private
+  @Unstable
+  public abstract void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests);
+}

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/Container.java

@@ -57,8 +57,8 @@ import org.apache.hadoop.yarn.util.Records;
  * start/stop containers.</p>
  * 
  * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
- * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
- * @see ContainerManagementProtocol#stopContainer(org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest)
+ * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
+ * @see ContainerManagementProtocol#stopContainers(org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest)
  */
 @Public
 @Stable
@@ -155,7 +155,7 @@ public abstract class Container implements Comparable<Container> {
    * <code>Container</code> includes the <code>ContainerToken</code>.</p>
    *
    * @see ApplicationMasterProtocol#allocate(org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest)
-   * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
+   * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
    *
    * @return <code>ContainerToken</code> for the container
    */

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

@@ -49,7 +49,7 @@ import org.apache.hadoop.yarn.util.Records;
  *   </ul>
  * </p>
  * 
- * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
+ * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
  */
 @Public
 @Stable
@@ -132,7 +132,7 @@ public abstract class ContainerLaunchContext {
   
   /**
    * <p>
-   * Get application-specific binary <em>service data</em>. This is a map keyed
+   * Set application-specific binary <em>service data</em>. This is a map keyed
    * by the name of each {@link AuxiliaryService} that is configured on a
    * NodeManager and value correspond to the application specific data targeted
    * for the keyed {@link AuxiliaryService}. All pre-existing Map entries are

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResource.java

@@ -37,7 +37,7 @@ import org.apache.hadoop.yarn.util.Records;
  * @see LocalResourceVisibility
  * @see ContainerLaunchContext
  * @see ApplicationSubmissionContext
- * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
+ * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
  */
 @Public
 @Stable

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceType.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
  * @see LocalResource
  * @see ContainerLaunchContext
  * @see ApplicationSubmissionContext
- * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
+ * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
  */
 @Public
 @Stable

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/LocalResourceVisibility.java

@@ -43,7 +43,7 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
  * @see LocalResource
  * @see ContainerLaunchContext
  * @see ApplicationSubmissionContext
- * @see ContainerManagementProtocol#startContainer(org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest)
+ * @see ContainerManagementProtocol#startContainers(org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest)
  */
 @Public
 @Stable

+ 93 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/SerializedException.java

@@ -0,0 +1,93 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.records;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.util.Records;
+
+@Private
+@Unstable
+public abstract class SerializedException {
+
+  @Private
+  @Unstable
+  public static SerializedException newInstance(Throwable e) {
+    SerializedException exception =
+        Records.newRecord(SerializedException.class);
+    exception.init(e);
+    return exception;
+  }
+
+  /**
+   * Constructs a new <code>SerializedException</code> with the specified detail
+   * message and cause.
+   */
+  @Private
+  @Unstable
+  public abstract void init(String message, Throwable cause);
+
+  /**
+   * Constructs a new <code>SerializedException</code> with the specified detail
+   * message.
+   */
+  @Private
+  @Unstable
+  public abstract void init(String message);
+
+  /**
+   * Constructs a new <code>SerializedException</code> with the specified cause.
+   */
+  @Private
+  @Unstable
+  public abstract void init(Throwable cause);
+
+  /**
+   * Get the detail message string of this exception.
+   * @return the detail message string of this exception.
+   */
+  @Private
+  @Unstable
+  public abstract String getMessage();
+
+  /**
+   * Get the backtrace of this exception. 
+   * @return the backtrace of this exception.
+   */
+  @Private
+  @Unstable
+  public abstract String getRemoteTrace();
+
+  /**
+   * Get the cause of this exception or null if the cause is nonexistent or
+   * unknown.
+   * @return the cause of this exception.
+   */
+  @Private
+  @Unstable
+  public abstract SerializedException getCause();
+
+  /**
+   * Deserialize the exception to a new Throwable. 
+   * @return the Throwable form of this serialized exception.
+   */
+  @Private
+  @Unstable
+  public abstract Throwable deSerialize();
+}

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/InvalidContainerException.java

@@ -19,12 +19,12 @@
 package org.apache.hadoop.yarn.exceptions;
 
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 
 /**
  * This exception is thrown by a NodeManager that is rejecting start-container
  * requests via
- * {@link ContainerManagementProtocol#startContainer(StartContainerRequest)}
+ * {@link ContainerManagementProtocol#startContainers(StartContainersRequest)}
  * for containers allocated by a previous instance of the RM.
  */
 public class InvalidContainerException extends YarnException {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/exceptions/NMNotYetReadyException.java

@@ -19,11 +19,11 @@
 package org.apache.hadoop.yarn.exceptions;
 
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 
 /**
  * This exception is thrown on
- * {@link ContainerManagementProtocol#startContainer(StartContainerRequest)} API
+ * {@link ContainerManagementProtocol#startContainers(StartContainersRequest)} API
  * when an NM starts from scratch but has not yet connected with RM.
  */
 public class NMNotYetReadyException extends YarnException {

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/ApplicationInitializationContext.java

@@ -25,7 +25,7 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 
@@ -70,7 +70,7 @@ public class ApplicationInitializationContext {
 
   /**
    * Get the data sent to the NodeManager via
-   * {@link ContainerManagementProtocol#startContainer(StartContainerRequest)}
+   * {@link ContainerManagementProtocol#startContainers(StartContainersRequest)}
    * as part of {@link ContainerLaunchContext#getServiceData()}
    * 
    * @return the servicesData for this application.

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/server/api/AuxiliaryService.java

@@ -24,8 +24,8 @@ import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Evolving;
 import org.apache.hadoop.service.AbstractService;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 
 /**
@@ -71,8 +71,8 @@ public abstract class AuxiliaryService extends AbstractService {
    * 
    * <p>
    * The information is passed along to applications via
-   * {@link StartContainerResponse#getAllServicesMetaData()} that is returned by
-   * {@link ContainerManagementProtocol#startContainer(StartContainerRequest)}
+   * {@link StartContainersResponse#getAllServicesMetaData()} that is returned by
+   * {@link ContainerManagementProtocol#startContainers(StartContainersRequest)}
    * </p>
    * 
    * @return meta-data for this service that should be made available to

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/proto/containermanagement_protocol.proto

@@ -30,7 +30,7 @@ option java_generate_equals_and_hash = true;
 import "yarn_service_protos.proto";
 
 service ContainerManagementProtocolService {
-  rpc startContainer(StartContainerRequestProto) returns (StartContainerResponseProto);
-  rpc stopContainer(StopContainerRequestProto) returns (StopContainerResponseProto);
-  rpc getContainerStatus(GetContainerStatusRequestProto) returns (GetContainerStatusResponseProto);
+  rpc startContainers(StartContainersRequestProto) returns (StartContainersResponseProto);
+  rpc stopContainers(StopContainersRequestProto) returns (StopContainersResponseProto);
+  rpc getContainerStatuses(GetContainerStatusesRequestProto) returns (GetContainerStatusesResponseProto);
 }

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

@@ -182,3 +182,37 @@ message GetContainerStatusRequestProto {
 message GetContainerStatusResponseProto {
   optional ContainerStatusProto status = 1;
 }
+
+//// bulk API records
+message StartContainersRequestProto {
+  repeated StartContainerRequestProto start_container_request = 1;
+}
+
+message ContainerExceptionMapProto {
+  optional ContainerIdProto container_id = 1;
+  optional SerializedExceptionProto exception = 2;
+}
+
+message StartContainersResponseProto {
+  repeated StringBytesMapProto services_meta_data = 1;
+  repeated ContainerIdProto succeeded_requests = 2;
+  repeated ContainerExceptionMapProto failed_requests = 3;
+}
+
+message StopContainersRequestProto {
+  repeated ContainerIdProto container_id = 1;
+}
+
+message StopContainersResponseProto {
+  repeated ContainerIdProto succeeded_requests = 1;
+  repeated ContainerExceptionMapProto failed_requests = 2;
+}
+
+message GetContainerStatusesRequestProto {
+  repeated ContainerIdProto container_id = 1;
+}
+
+message GetContainerStatusesResponseProto {
+  repeated ContainerStatusProto status = 1;
+  repeated ContainerExceptionMapProto failed_requests = 2;
+}

+ 59 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/main/java/org/apache/hadoop/yarn/client/api/impl/NMClientImpl.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.client.api.impl;
 
 import java.io.IOException;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.ConcurrentMap;
@@ -30,9 +32,14 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
@@ -181,11 +188,23 @@ public class NMClientImpl extends NMClient {
         proxy =
             cmProxy.getProxy(container.getNodeId().toString(),
                 container.getId());
-        allServiceResponse =
+        StartContainerRequest scRequest =
+            StartContainerRequest.newInstance(containerLaunchContext,
+              container.getContainerToken());
+        List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+        list.add(scRequest);
+        StartContainersRequest allRequests =
+            StartContainersRequest.newInstance(list);
+        StartContainersResponse response =
             proxy
-                .getContainerManagementProtocol().startContainer(
-                    StartContainerRequest.newInstance(containerLaunchContext,
-                        container.getContainerToken())).getAllServicesMetaData();
+                .getContainerManagementProtocol().startContainers(allRequests);
+        if (response.getFailedRequests() != null
+            && response.getFailedRequests().containsKey(container.getId())) {
+          Throwable t =
+              response.getFailedRequests().get(container.getId()).deSerialize();
+          parseAndThrowException(t);
+        }
+        allServiceResponse = response.getAllServicesMetaData();
         startingContainer.state = ContainerState.RUNNING;
       } catch (YarnException e) {
         startingContainer.state = ContainerState.COMPLETE;
@@ -238,11 +257,20 @@ public class NMClientImpl extends NMClient {
       NodeId nodeId) throws YarnException, IOException {
 
     ContainerManagementProtocolProxyData proxy = null;
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
     try {
       proxy = cmProxy.getProxy(nodeId.toString(), containerId);
-      ContainerStatus containerStatus =
-          proxy.getContainerManagementProtocol().getContainerStatus(
-              GetContainerStatusRequest.newInstance(containerId)).getStatus();
+      GetContainerStatusesResponse response =
+          proxy.getContainerManagementProtocol().getContainerStatuses(
+              GetContainerStatusesRequest.newInstance(containerIds));
+      if (response.getFailedRequests() != null
+          && response.getFailedRequests().containsKey(containerId)) {
+        Throwable t =
+            response.getFailedRequests().get(containerId).deSerialize();
+        parseAndThrowException(t);
+      }
+      ContainerStatus containerStatus = response.getContainerStatuses().get(0);
       return containerStatus;
     } finally {
       if (proxy != null) {
@@ -254,10 +282,19 @@ public class NMClientImpl extends NMClient {
   private void stopContainerInternal(ContainerId containerId, NodeId nodeId)
       throws IOException, YarnException {
     ContainerManagementProtocolProxyData proxy = null;
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
     try {
       proxy = cmProxy.getProxy(nodeId.toString(), containerId);
-      proxy.getContainerManagementProtocol().stopContainer(
-          StopContainerRequest.newInstance(containerId));
+      StopContainersResponse response =
+          proxy.getContainerManagementProtocol().stopContainers(
+            StopContainersRequest.newInstance(containerIds));
+      if (response.getFailedRequests() != null
+          && response.getFailedRequests().containsKey(containerId)) {
+        Throwable t = response.getFailedRequests().get(containerId)
+          .deSerialize();
+        parseAndThrowException(t);
+      }
     } finally {
       if (proxy != null) {
         cmProxy.mayBeCloseProxy(proxy);
@@ -285,4 +322,15 @@ public class NMClientImpl extends NMClient {
   public AtomicBoolean getCleanupRunningContainers() {
     return cleanupRunningContainers;
   }
+
+  private void parseAndThrowException(Throwable t) throws YarnException,
+      IOException {
+    if (t instanceof YarnException) {
+      throw (YarnException) t;
+    } else if (t instanceof InvalidToken) {
+      throw (InvalidToken) t;
+    } else {
+      throw (IOException) t;
+    }
+  }
 }

+ 32 - 32
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/client/ContainerManagementProtocolPBClientImpl.java

@@ -30,24 +30,24 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
 
 import com.google.protobuf.ServiceException;
 
@@ -87,14 +87,14 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
   }
 
   @Override
-  public GetContainerStatusResponse getContainerStatus(
-      GetContainerStatusRequest request) throws YarnException,
-      IOException {
-    GetContainerStatusRequestProto requestProto =
-        ((GetContainerStatusRequestPBImpl) request).getProto();
+  public StartContainersResponse
+      startContainers(StartContainersRequest requests) throws YarnException,
+          IOException {
+    StartContainersRequestProto requestProto =
+        ((StartContainersRequestPBImpl) requests).getProto();
     try {
-      return new GetContainerStatusResponsePBImpl(proxy.getContainerStatus(
-        null, requestProto));
+      return new StartContainersResponsePBImpl(proxy.startContainers(null,
+        requestProto));
     } catch (ServiceException e) {
       RPCUtil.unwrapAndThrowException(e);
       return null;
@@ -102,12 +102,12 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
   }
 
   @Override
-  public StartContainerResponse startContainer(StartContainerRequest request)
+  public StopContainersResponse stopContainers(StopContainersRequest requests)
       throws YarnException, IOException {
-    StartContainerRequestProto requestProto =
-        ((StartContainerRequestPBImpl) request).getProto();
+    StopContainersRequestProto requestProto =
+        ((StopContainersRequestPBImpl) requests).getProto();
     try {
-      return new StartContainerResponsePBImpl(proxy.startContainer(null,
+      return new StopContainersResponsePBImpl(proxy.stopContainers(null,
         requestProto));
     } catch (ServiceException e) {
       RPCUtil.unwrapAndThrowException(e);
@@ -116,13 +116,13 @@ public class ContainerManagementProtocolPBClientImpl implements ContainerManagem
   }
 
   @Override
-  public StopContainerResponse stopContainer(StopContainerRequest request)
-      throws YarnException, IOException {
-    StopContainerRequestProto requestProto =
-        ((StopContainerRequestPBImpl) request).getProto();
+  public GetContainerStatusesResponse getContainerStatuses(
+      GetContainerStatusesRequest request) throws YarnException, IOException {
+    GetContainerStatusesRequestProto requestProto =
+        ((GetContainerStatusesRequestPBImpl) request).getProto();
     try {
-      return new StopContainerResponsePBImpl(proxy.stopContainer(null,
-        requestProto));
+      return new GetContainerStatusesResponsePBImpl(proxy.getContainerStatuses(
+        null, requestProto));
     } catch (ServiceException e) {
       RPCUtil.unwrapAndThrowException(e);
       return null;

+ 31 - 31
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/impl/pb/service/ContainerManagementProtocolPBServiceImpl.java

@@ -23,22 +23,22 @@ import java.io.IOException;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainerResponsePBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerRequestPBImpl;
-import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainerResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.GetContainerStatusesResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StartContainersResponsePBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersRequestPBImpl;
+import org.apache.hadoop.yarn.api.protocolrecords.impl.pb.StopContainersResponsePBImpl;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
 
 import com.google.protobuf.RpcController;
 import com.google.protobuf.ServiceException;
@@ -53,12 +53,12 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
   }
 
   @Override
-  public GetContainerStatusResponseProto getContainerStatus(RpcController arg0,
-      GetContainerStatusRequestProto proto) throws ServiceException {
-    GetContainerStatusRequestPBImpl request = new GetContainerStatusRequestPBImpl(proto);
+  public StartContainersResponseProto startContainers(RpcController arg0,
+      StartContainersRequestProto proto) throws ServiceException {
+    StartContainersRequestPBImpl request = new StartContainersRequestPBImpl(proto);
     try {
-      GetContainerStatusResponse response = real.getContainerStatus(request);
-      return ((GetContainerStatusResponsePBImpl)response).getProto();
+      StartContainersResponse response = real.startContainers(request);
+      return ((StartContainersResponsePBImpl)response).getProto();
     } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
@@ -67,12 +67,12 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
   }
 
   @Override
-  public StartContainerResponseProto startContainer(RpcController arg0,
-      StartContainerRequestProto proto) throws ServiceException {
-    StartContainerRequestPBImpl request = new StartContainerRequestPBImpl(proto);
+  public StopContainersResponseProto stopContainers(RpcController arg0,
+      StopContainersRequestProto proto) throws ServiceException {
+    StopContainersRequestPBImpl request = new StopContainersRequestPBImpl(proto);
     try {
-      StartContainerResponse response = real.startContainer(request);
-      return ((StartContainerResponsePBImpl)response).getProto();
+      StopContainersResponse response = real.stopContainers(request);
+      return ((StopContainersResponsePBImpl)response).getProto();
     } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
@@ -81,17 +81,17 @@ public class ContainerManagementProtocolPBServiceImpl implements ContainerManage
   }
 
   @Override
-  public StopContainerResponseProto stopContainer(RpcController arg0,
-      StopContainerRequestProto proto) throws ServiceException {
-    StopContainerRequestPBImpl request = new StopContainerRequestPBImpl(proto);
+  public GetContainerStatusesResponseProto getContainerStatuses(
+      RpcController arg0, GetContainerStatusesRequestProto proto)
+      throws ServiceException {
+    GetContainerStatusesRequestPBImpl request = new GetContainerStatusesRequestPBImpl(proto);
     try {
-      StopContainerResponse response = real.stopContainer(request);
-      return ((StopContainerResponsePBImpl)response).getProto();
+      GetContainerStatusesResponse response = real.getContainerStatuses(request);
+      return ((GetContainerStatusesResponsePBImpl)response).getProto();
     } catch (YarnException e) {
       throw new ServiceException(e);
     } catch (IOException e) {
       throw new ServiceException(e);
     }
   }
-
 }

+ 0 - 130
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusResponsePBImpl.java

@@ -1,130 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
-
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
-import org.apache.hadoop.yarn.api.records.ContainerStatus;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusResponseProtoOrBuilder;
-
-@Private
-@Unstable
-public class GetContainerStatusResponsePBImpl extends  GetContainerStatusResponse {
-  GetContainerStatusResponseProto proto = GetContainerStatusResponseProto.getDefaultInstance();
-  GetContainerStatusResponseProto.Builder builder = null;
-  boolean viaProto = false;
-  
-  private ContainerStatus containerStatus = null;
-  
-  
-  public GetContainerStatusResponsePBImpl() {
-    builder = GetContainerStatusResponseProto.newBuilder();
-  }
-
-  public GetContainerStatusResponsePBImpl(GetContainerStatusResponseProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-  
-  public GetContainerStatusResponseProto getProto() {
-      mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  @Override
-  public int hashCode() {
-    return getProto().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null)
-      return false;
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      return this.getProto().equals(this.getClass().cast(other).getProto());
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
-  }
-
-  private void mergeLocalToBuilder() {
-    if (this.containerStatus != null) {
-      builder.setStatus(convertToProtoFormat(this.containerStatus));
-    }
-  }
-
-  private void mergeLocalToProto() {
-    if (viaProto) 
-      maybeInitBuilder();
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = GetContainerStatusResponseProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-    
-  
-  @Override
-  public ContainerStatus getStatus() {
-    GetContainerStatusResponseProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.containerStatus != null) {
-      return this.containerStatus;
-    }
-    if (!p.hasStatus()) {
-      return null;
-    }
-    this.containerStatus = convertFromProtoFormat(p.getStatus());
-    return this.containerStatus;
-  }
-
-  @Override
-  public void setStatus(ContainerStatus status) {
-    maybeInitBuilder();
-    if (status == null) 
-      builder.clearStatus();
-    this.containerStatus = status;
-  }
-
-  private ContainerStatusPBImpl convertFromProtoFormat(ContainerStatusProto p) {
-    return new ContainerStatusPBImpl(p);
-  }
-
-  private ContainerStatusProto convertToProtoFormat(ContainerStatus t) {
-    return ((ContainerStatusPBImpl)t).getProto();
-  }
-
-
-
-}  

+ 58 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusRequestPBImpl.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesRequestPBImpl.java

@@ -18,37 +18,41 @@
 
 package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
 
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusRequestProtoOrBuilder;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesRequestProtoOrBuilder;
 
 @Private
 @Unstable
-public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
-  GetContainerStatusRequestProto proto = GetContainerStatusRequestProto.getDefaultInstance();
-  GetContainerStatusRequestProto.Builder builder = null;
+public class GetContainerStatusesRequestPBImpl extends
+    GetContainerStatusesRequest {
+  GetContainerStatusesRequestProto proto = GetContainerStatusesRequestProto
+    .getDefaultInstance();
+  GetContainerStatusesRequestProto.Builder builder = null;
   boolean viaProto = false;
-  
-  private ContainerId containerId = null;
-  
-  
-  public GetContainerStatusRequestPBImpl() {
-    builder = GetContainerStatusRequestProto.newBuilder();
+
+  private List<ContainerId> containerIds = null;
+
+  public GetContainerStatusesRequestPBImpl() {
+    builder = GetContainerStatusesRequestProto.newBuilder();
   }
 
-  public GetContainerStatusRequestPBImpl(GetContainerStatusRequestProto proto) {
+  public GetContainerStatusesRequestPBImpl(
+      GetContainerStatusesRequestProto proto) {
     this.proto = proto;
     viaProto = true;
   }
-  
-  public GetContainerStatusRequestProto getProto() {
-      mergeLocalToProto();
+
+  public GetContainerStatusesRequestProto getProto() {
+    mergeLocalToProto();
     proto = viaProto ? proto : builder.build();
     viaProto = true;
     return proto;
@@ -71,17 +75,18 @@ public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
 
   @Override
   public String toString() {
-    return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
+    return getProto().toString().replaceAll("\\n", ", ")
+      .replaceAll("\\s+", " ");
   }
 
   private void mergeLocalToBuilder() {
-    if (this.containerId != null) {
-      builder.setContainerId(convertToProtoFormat(this.containerId));
+    if (this.containerIds != null) {
+      addLocalContainerIdsToProto();
     }
   }
 
   private void mergeLocalToProto() {
-    if (viaProto) 
+    if (viaProto)
       maybeInitBuilder();
     mergeLocalToBuilder();
     proto = builder.build();
@@ -90,31 +95,47 @@ public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
 
   private void maybeInitBuilder() {
     if (viaProto || builder == null) {
-      builder = GetContainerStatusRequestProto.newBuilder(proto);
+      builder = GetContainerStatusesRequestProto.newBuilder(proto);
     }
     viaProto = false;
   }
-    
-  
-  @Override
-  public ContainerId getContainerId() {
-    GetContainerStatusRequestProtoOrBuilder p = viaProto ? proto : builder;
-    if (this.containerId != null) {
-      return this.containerId;
+
+  private void addLocalContainerIdsToProto() {
+    maybeInitBuilder();
+    builder.clearContainerId();
+    if (this.containerIds == null)
+      return;
+    List<ContainerIdProto> protoList = new ArrayList<ContainerIdProto>();
+    for (ContainerId id : containerIds) {
+      protoList.add(convertToProtoFormat(id));
+    }
+    builder.addAllContainerId(protoList);
+  }
+
+  private void initLocalContainerIds() {
+    if (this.containerIds != null) {
+      return;
     }
-    if (!p.hasContainerId()) {
-      return null;
+    GetContainerStatusesRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerIdProto> containerIds = p.getContainerIdList();
+    this.containerIds = new ArrayList<ContainerId>();
+    for (ContainerIdProto id : containerIds) {
+      this.containerIds.add(convertFromProtoFormat(id));
     }
-    this.containerId = convertFromProtoFormat(p.getContainerId());
-    return this.containerId;
   }
 
   @Override
-  public void setContainerId(ContainerId containerId) {
+  public List<ContainerId> getContainerIds() {
+    initLocalContainerIds();
+    return this.containerIds;
+  }
+
+  @Override
+  public void setContainerIds(List<ContainerId> containerIds) {
     maybeInitBuilder();
-    if (containerId == null) 
+    if (containerIds == null)
       builder.clearContainerId();
-    this.containerId = containerId;
+    this.containerIds = containerIds;
   }
 
   private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
@@ -122,9 +143,7 @@ public class GetContainerStatusRequestPBImpl extends GetContainerStatusRequest {
   }
 
   private ContainerIdProto convertToProtoFormat(ContainerId t) {
-    return ((ContainerIdPBImpl)t).getProto();
+    return ((ContainerIdPBImpl) t).getProto();
   }
 
-
-
-}  
+}

+ 223 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/GetContainerStatusesResponsePBImpl.java

@@ -0,0 +1,223 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerStatusPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerStatusProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.GetContainerStatusesResponseProtoOrBuilder;
+
+@Private
+@Unstable
+public class GetContainerStatusesResponsePBImpl extends
+    GetContainerStatusesResponse {
+  GetContainerStatusesResponseProto proto = GetContainerStatusesResponseProto
+    .getDefaultInstance();
+  GetContainerStatusesResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private List<ContainerStatus> containerStatuses = null;
+  private Map<ContainerId, SerializedException> failedRequests = null;
+
+  public GetContainerStatusesResponsePBImpl() {
+    builder = GetContainerStatusesResponseProto.newBuilder();
+  }
+
+  public GetContainerStatusesResponsePBImpl(
+      GetContainerStatusesResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public GetContainerStatusesResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getProto().toString().replaceAll("\\n", ", ")
+      .replaceAll("\\s+", " ");
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.containerStatuses != null) {
+      addLocalContainerStatusesToProto();
+    }
+    if (this.failedRequests != null) {
+      addFailedRequestsToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = GetContainerStatusesResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void addLocalContainerStatusesToProto() {
+    maybeInitBuilder();
+    builder.clearStatus();
+    if (this.containerStatuses == null)
+      return;
+    List<ContainerStatusProto> protoList =
+        new ArrayList<ContainerStatusProto>();
+    for (ContainerStatus status : containerStatuses) {
+      protoList.add(convertToProtoFormat(status));
+    }
+    builder.addAllStatus(protoList);
+  }
+
+  private void addFailedRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearFailedRequests();
+    if (this.failedRequests == null)
+      return;
+    List<ContainerExceptionMapProto> protoList =
+        new ArrayList<ContainerExceptionMapProto>();
+    for (Map.Entry<ContainerId, SerializedException> entry : this.failedRequests
+      .entrySet()) {
+      protoList.add(ContainerExceptionMapProto.newBuilder()
+        .setContainerId(convertToProtoFormat(entry.getKey()))
+        .setException(convertToProtoFormat(entry.getValue())).build());
+    }
+    builder.addAllFailedRequests(protoList);
+  }
+
+  private void initLocalContainerStatuses() {
+    if (this.containerStatuses != null) {
+      return;
+    }
+    GetContainerStatusesResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerStatusProto> statuses = p.getStatusList();
+    this.containerStatuses = new ArrayList<ContainerStatus>();
+    for (ContainerStatusProto status : statuses) {
+      this.containerStatuses.add(convertFromProtoFormat(status));
+    }
+  }
+
+  private void initFailedRequests() {
+    if (this.failedRequests != null) {
+      return;
+    }
+    GetContainerStatusesResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
+    this.failedRequests = new HashMap<ContainerId, SerializedException>();
+    for (ContainerExceptionMapProto ce : protoList) {
+      this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()),
+        convertFromProtoFormat(ce.getException()));
+    }
+  }
+
+  @Override
+  public List<ContainerStatus> getContainerStatuses() {
+    initLocalContainerStatuses();
+    return this.containerStatuses;
+  }
+
+  @Override
+  public void setContainerStatuses(List<ContainerStatus> statuses) {
+    maybeInitBuilder();
+    if (statuses == null)
+      builder.clearStatus();
+    this.containerStatuses = statuses;
+  }
+
+  @Override
+  public Map<ContainerId, SerializedException> getFailedRequests() {
+    initFailedRequests();
+    return this.failedRequests;
+  }
+
+  @Override
+  public void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests) {
+    maybeInitBuilder();
+    if (failedRequests == null)
+      builder.clearFailedRequests();
+    this.failedRequests = failedRequests;
+  }
+
+  private ContainerStatusPBImpl convertFromProtoFormat(ContainerStatusProto p) {
+    return new ContainerStatusPBImpl(p);
+  }
+
+  private ContainerStatusProto convertToProtoFormat(ContainerStatus t) {
+    return ((ContainerStatusPBImpl) t).getProto();
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private SerializedExceptionPBImpl convertFromProtoFormat(
+      SerializedExceptionProto p) {
+    return new SerializedExceptionPBImpl(p);
+  }
+
+  private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
+    return ((SerializedExceptionPBImpl) t).getProto();
+  }
+}

+ 0 - 176
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainerResponsePBImpl.java

@@ -1,176 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
-
-
-import java.nio.ByteBuffer;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
-import org.apache.hadoop.yarn.proto.YarnProtos.StringBytesMapProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerResponseProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerResponseProtoOrBuilder;
-
-import com.google.protobuf.ByteString;
-
-@Private
-@Unstable
-public class StartContainerResponsePBImpl extends StartContainerResponse {
-  StartContainerResponseProto proto = StartContainerResponseProto.getDefaultInstance();
-  StartContainerResponseProto.Builder builder = null;
-  boolean viaProto = false;
- 
-  private Map<String, ByteBuffer> servicesMetaData = null;
-
-  public StartContainerResponsePBImpl() {
-    builder = StartContainerResponseProto.newBuilder();
-  }
-
-  public StartContainerResponsePBImpl(StartContainerResponseProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-  
-  public synchronized StartContainerResponseProto getProto() {
-    mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  @Override
-  public int hashCode() {
-    return getProto().hashCode();
-  }
-
-  @Override
-  public boolean equals(Object other) {
-    if (other == null)
-      return false;
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      return this.getProto().equals(this.getClass().cast(other).getProto());
-    }
-    return false;
-  }
-
-  @Override
-  public String toString() {
-    return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
-  }
-
-  private synchronized void mergeLocalToBuilder() {
-    if (this.servicesMetaData != null) {
-      addServicesMetaDataToProto();
-    }
-  }
-  
-  protected final ByteBuffer convertFromProtoFormat(ByteString byteString) {
-    return ProtoUtils.convertFromProtoFormat(byteString);
-  }
-
-  protected final ByteString convertToProtoFormat(ByteBuffer byteBuffer) {
-    return ProtoUtils.convertToProtoFormat(byteBuffer);
-  }
-
-  private synchronized void mergeLocalToProto() {
-    if (viaProto) {
-      maybeInitBuilder();
-    }
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private synchronized void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = StartContainerResponseProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-   
-
-  @Override
-  public synchronized Map<String, ByteBuffer> getAllServicesMetaData() {
-    initServicesMetaData();
-    return this.servicesMetaData;
-  }
-  @Override
-  public synchronized void setAllServicesMetaData(
-      Map<String, ByteBuffer> servicesMetaData) {
-    if(servicesMetaData == null) {
-      return;
-    }
-    initServicesMetaData();
-    this.servicesMetaData.clear();
-    this.servicesMetaData.putAll(servicesMetaData);
-  }
-  
-  private synchronized void initServicesMetaData() {
-    if (this.servicesMetaData != null) {
-      return;
-    }
-    StartContainerResponseProtoOrBuilder p = viaProto ? proto : builder;
-    List<StringBytesMapProto> list = p.getServicesMetaDataList();
-    this.servicesMetaData = new HashMap<String, ByteBuffer>();
-
-    for (StringBytesMapProto c : list) {
-      this.servicesMetaData.put(c.getKey(), convertFromProtoFormat(c.getValue()));
-    }
-  }
-  
-  private synchronized void addServicesMetaDataToProto() {
-    maybeInitBuilder();
-    builder.clearServicesMetaData();
-    if (servicesMetaData == null)
-      return;
-    Iterable<StringBytesMapProto> iterable = new Iterable<StringBytesMapProto>() {
-      
-      @Override
-      public synchronized Iterator<StringBytesMapProto> iterator() {
-        return new Iterator<StringBytesMapProto>() {
-          
-          Iterator<String> keyIter = servicesMetaData.keySet().iterator();
-          
-          @Override
-          public synchronized void remove() {
-            throw new UnsupportedOperationException();
-          }
-          
-          @Override
-          public synchronized StringBytesMapProto next() {
-            String key = keyIter.next();
-            return StringBytesMapProto.newBuilder().setKey(key).setValue(convertToProtoFormat(servicesMetaData.get(key))).build();
-          }
-          
-          @Override
-          public synchronized boolean hasNext() {
-            return keyIter.hasNext();
-          }
-        };
-      }
-    };
-    builder.addAllServicesMetaData(iterable);
-  }
-}  

+ 139 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersRequestPBImpl.java

@@ -0,0 +1,139 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainerRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersRequestProtoOrBuilder;
+
+public class StartContainersRequestPBImpl extends StartContainersRequest {
+  StartContainersRequestProto proto = StartContainersRequestProto
+    .getDefaultInstance();
+  StartContainersRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private List<StartContainerRequest> requests = null;
+
+  public StartContainersRequestPBImpl() {
+    builder = StartContainersRequestProto.newBuilder();
+  }
+
+  public StartContainersRequestPBImpl(StartContainersRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public StartContainersRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void mergeLocalToBuilder() {
+    if (requests != null) {
+      addLocalRequestsToProto();
+    }
+  }
+
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = StartContainersRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void addLocalRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearStartContainerRequest();
+    List<StartContainerRequestProto> protoList =
+        new ArrayList<StartContainerRequestProto>();
+    for (StartContainerRequest r : this.requests) {
+      protoList.add(convertToProtoFormat(r));
+    }
+    builder.addAllStartContainerRequest(protoList);
+  }
+
+  private void initLocalRequests() {
+    StartContainersRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<StartContainerRequestProto> requestList =
+        p.getStartContainerRequestList();
+    this.requests = new ArrayList<StartContainerRequest>();
+    for (StartContainerRequestProto r : requestList) {
+      this.requests.add(convertFromProtoFormat(r));
+    }
+  }
+
+  @Override
+  public void setStartContainerRequests(List<StartContainerRequest> requests) {
+    maybeInitBuilder();
+    if (requests == null) {
+      builder.clearStartContainerRequest();
+    }
+    this.requests = requests;
+  }
+
+  @Override
+  public List<StartContainerRequest> getStartContainerRequests() {
+    if (this.requests != null) {
+      return this.requests;
+    }
+    initLocalRequests();
+    return this.requests;
+  }
+
+  private StartContainerRequestPBImpl convertFromProtoFormat(
+      StartContainerRequestProto p) {
+    return new StartContainerRequestPBImpl(p);
+  }
+
+  private StartContainerRequestProto convertToProtoFormat(
+      StartContainerRequest t) {
+    return ((StartContainerRequestPBImpl) t).getProto();
+  }
+}

+ 319 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StartContainersResponsePBImpl.java

@@ -0,0 +1,319 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.StringBytesMapProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StartContainersResponseProtoOrBuilder;
+
+import com.google.protobuf.ByteString;
+
+@Private
+@Unstable
+public class StartContainersResponsePBImpl extends StartContainersResponse {
+  StartContainersResponseProto proto = StartContainersResponseProto
+    .getDefaultInstance();
+  StartContainersResponseProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private Map<String, ByteBuffer> servicesMetaData = null;
+  private List<ContainerId> succeededContainers = null;
+  private Map<ContainerId, SerializedException> failedContainers = null;
+
+  public StartContainersResponsePBImpl() {
+    builder = StartContainersResponseProto.newBuilder();
+  }
+
+  public StartContainersResponsePBImpl(StartContainersResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public StartContainersResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getProto().toString().replaceAll("\\n", ", ")
+      .replaceAll("\\s+", " ");
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.servicesMetaData != null) {
+      addServicesMetaDataToProto();
+    }
+    if (this.succeededContainers != null) {
+      addSucceededContainersToProto();
+    }
+    if (this.failedContainers != null) {
+      addFailedContainersToProto();
+    }
+  }
+
+  protected final ByteBuffer convertFromProtoFormat(ByteString byteString) {
+    return ProtoUtils.convertFromProtoFormat(byteString);
+  }
+
+  protected final ByteString convertToProtoFormat(ByteBuffer byteBuffer) {
+    return ProtoUtils.convertToProtoFormat(byteBuffer);
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private SerializedExceptionPBImpl convertFromProtoFormat(
+      SerializedExceptionProto p) {
+    return new SerializedExceptionPBImpl(p);
+  }
+
+  private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
+    return ((SerializedExceptionPBImpl) t).getProto();
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = StartContainersResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  @Override
+  public Map<String, ByteBuffer> getAllServicesMetaData() {
+    initServicesMetaData();
+    return this.servicesMetaData;
+  }
+
+  @Override
+  public void setAllServicesMetaData(Map<String, ByteBuffer> servicesMetaData) {
+    if (servicesMetaData == null) {
+      return;
+    }
+    initServicesMetaData();
+    this.servicesMetaData.clear();
+    this.servicesMetaData.putAll(servicesMetaData);
+  }
+
+  private void initServicesMetaData() {
+    if (this.servicesMetaData != null) {
+      return;
+    }
+    StartContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<StringBytesMapProto> list = p.getServicesMetaDataList();
+    this.servicesMetaData = new HashMap<String, ByteBuffer>();
+
+    for (StringBytesMapProto c : list) {
+      this.servicesMetaData.put(c.getKey(),
+        convertFromProtoFormat(c.getValue()));
+    }
+  }
+
+  private void addServicesMetaDataToProto() {
+    maybeInitBuilder();
+    builder.clearServicesMetaData();
+    if (servicesMetaData == null)
+      return;
+    Iterable<StringBytesMapProto> iterable =
+        new Iterable<StringBytesMapProto>() {
+
+          @Override
+          public Iterator<StringBytesMapProto> iterator() {
+            return new Iterator<StringBytesMapProto>() {
+
+              Iterator<String> keyIter = servicesMetaData.keySet().iterator();
+
+              @Override
+              public void remove() {
+                throw new UnsupportedOperationException();
+              }
+
+              @Override
+              public StringBytesMapProto next() {
+                String key = keyIter.next();
+                return StringBytesMapProto.newBuilder().setKey(key)
+                  .setValue(convertToProtoFormat(servicesMetaData.get(key)))
+                  .build();
+              }
+
+              @Override
+              public boolean hasNext() {
+                return keyIter.hasNext();
+              }
+            };
+          }
+        };
+    builder.addAllServicesMetaData(iterable);
+  }
+
+  private void addFailedContainersToProto() {
+    maybeInitBuilder();
+    builder.clearFailedRequests();
+    if (this.failedContainers == null)
+      return;
+    List<ContainerExceptionMapProto> protoList =
+        new ArrayList<ContainerExceptionMapProto>();
+
+    for (Map.Entry<ContainerId, SerializedException> entry : this.failedContainers
+      .entrySet()) {
+      protoList.add(ContainerExceptionMapProto.newBuilder()
+        .setContainerId(convertToProtoFormat(entry.getKey()))
+        .setException(convertToProtoFormat(entry.getValue())).build());
+    }
+    builder.addAllFailedRequests(protoList);
+  }
+
+  private void addSucceededContainersToProto() {
+    maybeInitBuilder();
+    builder.clearSucceededRequests();
+    if (this.succeededContainers == null) {
+      return;
+    }
+    Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
+      @Override
+      public Iterator<ContainerIdProto> iterator() {
+        return new Iterator<ContainerIdProto>() {
+
+          Iterator<ContainerId> iter = succeededContainers.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public ContainerIdProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+      }
+    };
+    builder.addAllSucceededRequests(iterable);
+  }
+
+  private void initSucceededContainers() {
+    if (this.succeededContainers != null)
+      return;
+    StartContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerIdProto> list = p.getSucceededRequestsList();
+    this.succeededContainers = new ArrayList<ContainerId>();
+    for (ContainerIdProto c : list) {
+      this.succeededContainers.add(convertFromProtoFormat(c));
+    }
+  }
+
+  @Override
+  public List<ContainerId> getSuccessfullyStartedContainers() {
+    initSucceededContainers();
+    return this.succeededContainers;
+  }
+
+  @Override
+  public void setSuccessfullyStartedContainers(
+      List<ContainerId> succeededContainers) {
+    maybeInitBuilder();
+    if (succeededContainers == null) {
+      builder.clearSucceededRequests();
+    }
+    this.succeededContainers = succeededContainers;
+  }
+
+  private void initFailedContainers() {
+    if (this.failedContainers != null) {
+      return;
+    }
+    StartContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
+    this.failedContainers = new HashMap<ContainerId, SerializedException>();
+    for (ContainerExceptionMapProto ce : protoList) {
+      this.failedContainers.put(convertFromProtoFormat(ce.getContainerId()),
+        convertFromProtoFormat(ce.getException()));
+    }
+  }
+
+  @Override
+  public Map<ContainerId, SerializedException> getFailedRequests() {
+    initFailedContainers();
+    return this.failedContainers;
+  }
+
+  @Override
+  public void setFailedRequests(
+      Map<ContainerId, SerializedException> failedContainers) {
+    maybeInitBuilder();
+    if (failedContainers == null)
+      builder.clearFailedRequests();
+    this.failedContainers = failedContainers;
+  }
+}

+ 0 - 130
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainerRequestPBImpl.java

@@ -1,130 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
-
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
-import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProto;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerRequestProtoOrBuilder;
-
-@Private
-@Unstable
-public class StopContainerRequestPBImpl extends StopContainerRequest {
-  StopContainerRequestProto proto = StopContainerRequestProto.getDefaultInstance();
-  StopContainerRequestProto.Builder builder = null;
-  boolean viaProto = false;
-  
-  private ContainerId containerId = null;
-  
-  
-  public StopContainerRequestPBImpl() {
-    builder = StopContainerRequestProto.newBuilder();
-  }
-
-  public StopContainerRequestPBImpl(StopContainerRequestProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-  
-  public StopContainerRequestProto getProto() {
-      mergeLocalToProto();
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-
-  @Override
-  public int hashCode() {
-    return getProto().hashCode();
-  }
-  
-  @Override
-  public boolean equals(Object other) {
-    if (other == null)
-      return false;
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      return this.getProto().equals(this.getClass().cast(other).getProto());
-    }
-    return false;
-  }
-  
-  @Override
-  public String toString() {
-    return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
-  }
-
-  private void mergeLocalToBuilder() {
-    if (this.containerId != null) {
-      builder.setContainerId(convertToProtoFormat(this.containerId));
-    }
-  }
-
-  private void mergeLocalToProto() {
-    if (viaProto) 
-      maybeInitBuilder();
-    mergeLocalToBuilder();
-    proto = builder.build();
-    viaProto = true;
-  }
-
-  private void maybeInitBuilder() {
-    if (viaProto || builder == null) {
-      builder = StopContainerRequestProto.newBuilder(proto);
-    }
-    viaProto = false;
-  }
-    
-  
-  @Override
-  public ContainerId getContainerId() {
-    StopContainerRequestProtoOrBuilder 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;
-  }
-
-  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
-    return new ContainerIdPBImpl(p);
-  }
-
-  private ContainerIdProto convertToProtoFormat(ContainerId t) {
-    return ((ContainerIdPBImpl)t).getProto();
-  }
-
-
-
-}  

+ 0 - 68
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainerResponsePBImpl.java

@@ -1,68 +0,0 @@
-/**
-* Licensed to the Apache Software Foundation (ASF) under one
-* or more contributor license agreements.  See the NOTICE file
-* distributed with this work for additional information
-* regarding copyright ownership.  The ASF licenses this file
-* to you under the Apache License, Version 2.0 (the
-* "License"); you may not use this file except in compliance
-* with the License.  You may obtain a copy of the License at
-*
-*     http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
-
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
-import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainerResponseProto;
-
-@Private
-@Unstable
-public class StopContainerResponsePBImpl extends StopContainerResponse {
-  StopContainerResponseProto proto = StopContainerResponseProto.getDefaultInstance();
-  StopContainerResponseProto.Builder builder = null;
-  boolean viaProto = false;
-  
-  public StopContainerResponsePBImpl() {
-    builder = StopContainerResponseProto.newBuilder();
-  }
-
-  public StopContainerResponsePBImpl(StopContainerResponseProto proto) {
-    this.proto = proto;
-    viaProto = true;
-  }
-  
-  public StopContainerResponseProto getProto() {
-    proto = viaProto ? proto : builder.build();
-    viaProto = true;
-    return proto;
-  }
-  
-  @Override
-  public int hashCode() {
-    return getProto().hashCode();
-  }
-  
-  @Override
-  public boolean equals(Object other) {
-    if (other == null)
-      return false;
-    if (other.getClass().isAssignableFrom(this.getClass())) {
-      return this.getProto().equals(this.getClass().cast(other).getProto());
-    }
-    return false;
-  }
-  
-  @Override
-  public String toString() {
-    return getProto().toString().replaceAll("\\n", ", ").replaceAll("\\s+", " ");
-  }
-}  

+ 146 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersRequestPBImpl.java

@@ -0,0 +1,146 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersRequestProtoOrBuilder;
+
+@Private
+@Unstable
+public class StopContainersRequestPBImpl extends StopContainersRequest {
+  StopContainersRequestProto proto = StopContainersRequestProto
+    .getDefaultInstance();
+  StopContainersRequestProto.Builder builder = null;
+  boolean viaProto = false;
+
+  private List<ContainerId> containerIds = null;
+
+  public StopContainersRequestPBImpl() {
+    builder = StopContainersRequestProto.newBuilder();
+  }
+
+  public StopContainersRequestPBImpl(StopContainersRequestProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public StopContainersRequestProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getProto().toString().replaceAll("\\n", ", ")
+      .replaceAll("\\s+", " ");
+  }
+
+  private void mergeLocalToBuilder() {
+    if (this.containerIds != null) {
+      addLocalContainerIdsToProto();
+    }
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto)
+      maybeInitBuilder();
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = StopContainersRequestProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void addLocalContainerIdsToProto() {
+    maybeInitBuilder();
+    builder.clearContainerId();
+    if (this.containerIds == null)
+      return;
+    List<ContainerIdProto> protoList = new ArrayList<ContainerIdProto>();
+    for (ContainerId id : containerIds) {
+      protoList.add(convertToProtoFormat(id));
+    }
+    builder.addAllContainerId(protoList);
+  }
+
+  private void initLocalContainerIds() {
+    if (this.containerIds != null) {
+      return;
+    }
+    StopContainersRequestProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerIdProto> containerIds = p.getContainerIdList();
+    this.containerIds = new ArrayList<ContainerId>();
+    for (ContainerIdProto id : containerIds) {
+      this.containerIds.add(convertFromProtoFormat(id));
+    }
+  }
+
+  @Override
+  public List<ContainerId> getContainerIds() {
+    initLocalContainerIds();
+    return this.containerIds;
+  }
+
+  @Override
+  public void setContainerIds(List<ContainerId> containerIds) {
+    maybeInitBuilder();
+    if (containerIds == null)
+      builder.clearContainerId();
+    this.containerIds = containerIds;
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+}

+ 234 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/protocolrecords/impl/pb/StopContainersResponsePBImpl.java

@@ -0,0 +1,234 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.yarn.api.protocolrecords.impl.pb;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.hadoop.classification.InterfaceAudience.Private;
+import org.apache.hadoop.classification.InterfaceStability.Unstable;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.api.records.impl.pb.ContainerIdPBImpl;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
+import org.apache.hadoop.yarn.proto.YarnProtos.ContainerIdProto;
+import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.ContainerExceptionMapProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProto;
+import org.apache.hadoop.yarn.proto.YarnServiceProtos.StopContainersResponseProtoOrBuilder;
+
+@Private
+@Unstable
+public class StopContainersResponsePBImpl extends StopContainersResponse {
+  StopContainersResponseProto proto = StopContainersResponseProto
+    .getDefaultInstance();
+  StopContainersResponseProto.Builder builder = null;
+  boolean viaProto = false;
+  private List<ContainerId> succeededRequests = null;
+  private Map<ContainerId, SerializedException> failedRequests = null;
+
+  public StopContainersResponsePBImpl() {
+    builder = StopContainersResponseProto.newBuilder();
+  }
+
+  public StopContainersResponsePBImpl(StopContainersResponseProto proto) {
+    this.proto = proto;
+    viaProto = true;
+  }
+
+  public StopContainersResponseProto getProto() {
+    mergeLocalToProto();
+    proto = viaProto ? proto : builder.build();
+    viaProto = true;
+    return proto;
+  }
+
+  @Override
+  public int hashCode() {
+    return getProto().hashCode();
+  }
+
+  @Override
+  public boolean equals(Object other) {
+    if (other == null)
+      return false;
+    if (other.getClass().isAssignableFrom(this.getClass())) {
+      return this.getProto().equals(this.getClass().cast(other).getProto());
+    }
+    return false;
+  }
+
+  @Override
+  public String toString() {
+    return getProto().toString().replaceAll("\\n", ", ")
+      .replaceAll("\\s+", " ");
+  }
+
+  private void mergeLocalToProto() {
+    if (viaProto) {
+      maybeInitBuilder();
+    }
+    mergeLocalToBuilder();
+    proto = builder.build();
+    viaProto = true;
+  }
+
+  private void maybeInitBuilder() {
+    if (viaProto || builder == null) {
+      builder = StopContainersResponseProto.newBuilder(proto);
+    }
+    viaProto = false;
+  }
+
+  private void mergeLocalToBuilder() {
+
+    if (this.succeededRequests != null) {
+      addSucceededRequestsToProto();
+    }
+    if (this.failedRequests != null) {
+      addFailedRequestsToProto();
+    }
+  }
+
+  private void addSucceededRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearSucceededRequests();
+    if (this.succeededRequests == null) {
+      return;
+    }
+    Iterable<ContainerIdProto> iterable = new Iterable<ContainerIdProto>() {
+      @Override
+      public Iterator<ContainerIdProto> iterator() {
+        return new Iterator<ContainerIdProto>() {
+
+          Iterator<ContainerId> iter = succeededRequests.iterator();
+
+          @Override
+          public boolean hasNext() {
+            return iter.hasNext();
+          }
+
+          @Override
+          public ContainerIdProto next() {
+            return convertToProtoFormat(iter.next());
+          }
+
+          @Override
+          public void remove() {
+            throw new UnsupportedOperationException();
+
+          }
+        };
+      }
+    };
+    builder.addAllSucceededRequests(iterable);
+  }
+
+  private void addFailedRequestsToProto() {
+    maybeInitBuilder();
+    builder.clearFailedRequests();
+    if (this.failedRequests == null)
+      return;
+    List<ContainerExceptionMapProto> protoList =
+        new ArrayList<ContainerExceptionMapProto>();
+
+    for (Map.Entry<ContainerId, SerializedException> entry : this.failedRequests
+      .entrySet()) {
+      protoList.add(ContainerExceptionMapProto.newBuilder()
+        .setContainerId(convertToProtoFormat(entry.getKey()))
+        .setException(convertToProtoFormat(entry.getValue())).build());
+    }
+    builder.addAllFailedRequests(protoList);
+  }
+
+  private void initSucceededRequests() {
+    if (this.succeededRequests != null)
+      return;
+    StopContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerIdProto> list = p.getSucceededRequestsList();
+    this.succeededRequests = new ArrayList<ContainerId>();
+    for (ContainerIdProto c : list) {
+      this.succeededRequests.add(convertFromProtoFormat(c));
+    }
+  }
+
+  private void initFailedRequests() {
+    if (this.failedRequests != null) {
+      return;
+    }
+    StopContainersResponseProtoOrBuilder p = viaProto ? proto : builder;
+    List<ContainerExceptionMapProto> protoList = p.getFailedRequestsList();
+    this.failedRequests = new HashMap<ContainerId, SerializedException>();
+    for (ContainerExceptionMapProto ce : protoList) {
+      this.failedRequests.put(convertFromProtoFormat(ce.getContainerId()),
+        convertFromProtoFormat(ce.getException()));
+    }
+  }
+
+  @Override
+  public List<ContainerId> getSuccessfullyStoppedContainers() {
+    initSucceededRequests();
+    return this.succeededRequests;
+  }
+
+  @Override
+  public void setSuccessfullyStoppedContainers(List<ContainerId> succeededRequests) {
+    maybeInitBuilder();
+    if (succeededRequests == null) {
+      builder.clearSucceededRequests();
+    }
+    this.succeededRequests = succeededRequests;
+  }
+
+  @Override
+  public Map<ContainerId, SerializedException> getFailedRequests() {
+    initFailedRequests();
+    return this.failedRequests;
+  }
+
+  @Override
+  public void setFailedRequests(
+      Map<ContainerId, SerializedException> failedRequests) {
+    maybeInitBuilder();
+    if (failedRequests == null)
+      builder.clearFailedRequests();
+    this.failedRequests = failedRequests;
+  }
+
+  private ContainerIdPBImpl convertFromProtoFormat(ContainerIdProto p) {
+    return new ContainerIdPBImpl(p);
+  }
+
+  private ContainerIdProto convertToProtoFormat(ContainerId t) {
+    return ((ContainerIdPBImpl) t).getProto();
+  }
+
+  private SerializedExceptionPBImpl convertFromProtoFormat(
+      SerializedExceptionProto p) {
+    return new SerializedExceptionPBImpl(p);
+  }
+
+  private SerializedExceptionProto convertToProtoFormat(SerializedException t) {
+    return ((SerializedExceptionPBImpl) t).getProto();
+  }
+}

+ 59 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/impl/pb/SerializedExceptionPBImpl.java → hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/SerializedExceptionPBImpl.java

@@ -16,14 +16,19 @@
  * limitations under the License.
  */
 
-package org.apache.hadoop.yarn.server.api.records.impl.pb;
+package org.apache.hadoop.yarn.api.records.impl.pb;
 
+import java.io.IOException;
 import java.io.PrintWriter;
 import java.io.StringWriter;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
 
+import org.apache.hadoop.yarn.api.records.SerializedException;
+import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.exceptions.YarnRuntimeException;
 import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProtoOrBuilder;
-import org.apache.hadoop.yarn.server.api.records.SerializedException;
 
 public class SerializedExceptionPBImpl extends SerializedException {
 
@@ -58,7 +63,6 @@ public class SerializedExceptionPBImpl extends SerializedException {
     if (t.getCause() == null) {
     } else {
       builder.setCause(new SerializedExceptionPBImpl(t.getCause()).getProto());
-      builder.setClassName(t.getClass().getCanonicalName());
     }
     StringWriter sw = new StringWriter();
     PrintWriter pw = new PrintWriter(sw);
@@ -68,6 +72,7 @@ public class SerializedExceptionPBImpl extends SerializedException {
       builder.setTrace(sw.toString());
     if (t.getMessage() != null)
       builder.setMessage(t.getMessage());
+    builder.setClassName(t.getClass().getCanonicalName());
   }
 
   public void init(String message, Throwable t) {
@@ -76,6 +81,32 @@ public class SerializedExceptionPBImpl extends SerializedException {
       builder.setMessage(message);
   }
 
+  @SuppressWarnings("unchecked")
+  @Override
+  public Throwable deSerialize() {
+
+    SerializedException cause = getCause();
+    SerializedExceptionProtoOrBuilder p = viaProto ? proto : builder;
+    Class<?> realClass = null;
+    try {
+      realClass = Class.forName(p.getClassName());
+    } catch (ClassNotFoundException e) {
+      throw new YarnRuntimeException(e);
+    }
+    Class classType = null;
+    if (YarnException.class.isAssignableFrom(realClass)) {
+      classType = YarnException.class;
+    } else if (IOException.class.isAssignableFrom(realClass)) {
+      classType = IOException.class;
+    } else if (RuntimeException.class.isAssignableFrom(realClass)) {
+      classType = RuntimeException.class;
+    } else {
+      classType = Exception.class;
+    }
+    return instantiateException(realClass.asSubclass(classType), getMessage(),
+      cause == null ? null : cause.deSerialize());
+  }
+
   @Override
   public String getMessage() {
     SerializedExceptionProtoOrBuilder p = viaProto ? proto : builder;
@@ -110,4 +141,29 @@ public class SerializedExceptionPBImpl extends SerializedException {
     }
     viaProto = false;
   }
+
+  private static <T extends Throwable> T instantiateException(
+      Class<? extends T> cls, String message, Throwable cause) {
+    Constructor<? extends T> cn;
+    T ex = null;
+    try {
+      cn = cls.getConstructor(String.class);
+      cn.setAccessible(true);
+      ex = cn.newInstance(message);
+      ex.initCause(cause);
+    } catch (SecurityException e) {
+      throw new YarnRuntimeException(e);
+    } catch (NoSuchMethodException e) {
+      throw new YarnRuntimeException(e);
+    } catch (IllegalArgumentException e) {
+      throw new YarnRuntimeException(e);
+    } catch (InstantiationException e) {
+      throw new YarnRuntimeException(e);
+    } catch (IllegalAccessException e) {
+      throw new YarnRuntimeException(e);
+    } catch (InvocationTargetException e) {
+      throw new YarnRuntimeException(e);
+    }
+    return ex;
+  }
 }

+ 31 - 23
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java

@@ -21,6 +21,8 @@ package org.apache.hadoop.yarn;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.net.SocketTimeoutException;
+import java.util.ArrayList;
+import java.util.List;
 
 import junit.framework.Assert;
 
@@ -31,12 +33,13 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -104,12 +107,15 @@ public class TestContainerLaunchRPC {
           TestRPC.newContainerToken(nodeId, "password".getBytes(),
             containerTokenIdentifier);
 
-      StartContainerRequest scRequest = recordFactory
-          .newRecordInstance(StartContainerRequest.class);
-      scRequest.setContainerLaunchContext(containerLaunchContext);
-      scRequest.setContainerToken(containerToken);
+      StartContainerRequest scRequest =
+          StartContainerRequest.newInstance(containerLaunchContext,
+            containerToken);
+      List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+      list.add(scRequest);
+      StartContainersRequest allRequests =
+          StartContainersRequest.newInstance(list);
       try {
-        proxy.startContainer(scRequest);
+        proxy.startContainers(allRequests);
       } catch (Exception e) {
         LOG.info(StringUtils.stringifyException(e));
         Assert.assertEquals("Error, exception is not: "
@@ -129,17 +135,8 @@ public class TestContainerLaunchRPC {
     private ContainerStatus status = null;
 
     @Override
-    public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request) throws YarnException {
-      GetContainerStatusResponse response = recordFactory
-          .newRecordInstance(GetContainerStatusResponse.class);
-      response.setStatus(status);
-      return response;
-    }
-
-    @Override
-    public StartContainerResponse startContainer(StartContainerRequest request)
-        throws YarnException, IOException {
+    public StartContainersResponse startContainers(
+        StartContainersRequest requests) throws YarnException, IOException {
       try {
         // make the thread sleep to look like its not going to respond
         Thread.sleep(10000);
@@ -151,11 +148,22 @@ public class TestContainerLaunchRPC {
     }
 
     @Override
-    public StopContainerResponse stopContainer(StopContainerRequest request)
-        throws YarnException {
+    public StopContainersResponse
+        stopContainers(StopContainersRequest requests) throws YarnException,
+            IOException {
       Exception e = new Exception("Dummy function", new Exception(
           "Dummy function cause"));
       throw new YarnException(e);
     }
+
+    @Override
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request) throws YarnException, IOException {
+      List<ContainerStatus> list = new ArrayList<ContainerStatus>();
+      list.add(status);
+      GetContainerStatusesResponse response =
+          GetContainerStatusesResponse.newInstance(list, null);
+      return null;
+    }
   }
 }

+ 61 - 46
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
 
 import junit.framework.Assert;
 
@@ -33,13 +35,14 @@ import org.apache.hadoop.security.SecurityUtil;
 import org.apache.hadoop.yarn.api.ApplicationClientProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocolPB;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.GetNewApplicationRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -122,9 +125,6 @@ public class TestRPC {
         ApplicationAttemptId.newInstance(applicationId, 0);
     ContainerId containerId =
         ContainerId.newInstance(applicationAttemptId, 100);
-    StartContainerRequest scRequest =
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    scRequest.setContainerLaunchContext(containerLaunchContext);
     NodeId nodeId = NodeId.newInstance("localhost", 1234);
     Resource resource = Resource.newInstance(1234, 2);
     ContainerTokenIdentifier containerTokenIdentifier =
@@ -132,22 +132,32 @@ public class TestRPC {
           resource, System.currentTimeMillis() + 10000, 42, 42);
     Token containerToken = newContainerToken(nodeId, "password".getBytes(),
           containerTokenIdentifier);
-    scRequest.setContainerToken(containerToken);
-    proxy.startContainer(scRequest);
-    
-    GetContainerStatusRequest gcsRequest = 
-        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-    gcsRequest.setContainerId(containerId);
-    GetContainerStatusResponse response =  proxy.getContainerStatus(gcsRequest);
-    ContainerStatus status = response.getStatus();
-    
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          containerToken);
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    proxy.startContainers(allRequests);
+
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    GetContainerStatusesResponse response =
+        proxy.getContainerStatuses(gcsRequest);
+    List<ContainerStatus> statuses = response.getContainerStatuses();
+
     //test remote exception
     boolean exception = false;
     try {
-      StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
-      stopRequest.setContainerId(containerId);
-      proxy.stopContainer(stopRequest);
-    } catch (YarnException e) {
+      StopContainersRequest stopRequest =
+          recordFactory.newRecordInstance(StopContainersRequest.class);
+      stopRequest.setContainerIds(containerIds);
+      proxy.stopContainers(stopRequest);
+      } catch (YarnException e) {
       exception = true;
       Assert.assertTrue(e.getMessage().contains(EXCEPTION_MSG));
       Assert.assertTrue(e.getMessage().contains(EXCEPTION_CAUSE));
@@ -158,46 +168,51 @@ public class TestRPC {
     Assert.assertTrue(exception);
     
     server.stop();
-    Assert.assertNotNull(status);
-    Assert.assertEquals(ContainerState.RUNNING, status.getState());
+    Assert.assertNotNull(statuses.get(0));
+    Assert.assertEquals(ContainerState.RUNNING, statuses.get(0).getState());
   }
 
   public class DummyContainerManager implements ContainerManagementProtocol {
 
-    private ContainerStatus status = null;    
-    
+    private List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
+
     @Override
-    public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request)
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request)
     throws YarnException {
-      GetContainerStatusResponse response = 
-          recordFactory.newRecordInstance(GetContainerStatusResponse.class);
-      response.setStatus(status);
+      GetContainerStatusesResponse response = 
+          recordFactory.newRecordInstance(GetContainerStatusesResponse.class);
+      response.setContainerStatuses(statuses);
       return response;
     }
 
     @Override
-    public StartContainerResponse startContainer(StartContainerRequest request) 
-        throws YarnException {
-      Token containerToken = request.getContainerToken();
-      ContainerTokenIdentifier tokenId = null;
-
-      try {
-        tokenId = newContainerTokenIdentifier(containerToken);
-      } catch (IOException e) {
-        throw RPCUtil.getRemoteException(e);
+    public StartContainersResponse startContainers(
+        StartContainersRequest requests) throws YarnException {
+      StartContainersResponse response =
+          recordFactory.newRecordInstance(StartContainersResponse.class);
+      for (StartContainerRequest request : requests.getStartContainerRequests()) {
+        Token containerToken = request.getContainerToken();
+        ContainerTokenIdentifier tokenId = null;
+
+        try {
+          tokenId = newContainerTokenIdentifier(containerToken);
+        } catch (IOException e) {
+          throw RPCUtil.getRemoteException(e);
+        }
+        ContainerStatus status =
+            recordFactory.newRecordInstance(ContainerStatus.class);
+        status.setState(ContainerState.RUNNING);
+        status.setContainerId(tokenId.getContainerID());
+        status.setExitStatus(0);
+        statuses.add(status);
+
       }
-      StartContainerResponse response = 
-          recordFactory.newRecordInstance(StartContainerResponse.class);
-      status = recordFactory.newRecordInstance(ContainerStatus.class);
-      status.setState(ContainerState.RUNNING);
-      status.setContainerId(tokenId.getContainerID());
-      status.setExitStatus(0);
       return response;
     }
 
     @Override
-    public StopContainerResponse stopContainer(StopContainerRequest request) 
+    public StopContainersResponse stopContainers(StopContainersRequest request) 
     throws YarnException {
       Exception e = new Exception(EXCEPTION_MSG, 
           new Exception(EXCEPTION_CAUSE));

+ 0 - 39
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/api/records/SerializedException.java

@@ -1,39 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.hadoop.yarn.server.api.records;
-
-import org.apache.hadoop.classification.InterfaceAudience.Private;
-import org.apache.hadoop.classification.InterfaceStability.Unstable;
-
-@Private
-@Unstable
-public abstract class SerializedException {
-  
-  public abstract void init(String message, Throwable cause);
-  
-  public abstract void init(String message);
-  
-  public abstract void init(Throwable cause);
-  
-  public abstract String getMessage();
-  
-  public abstract String getRemoteTrace();
-  
-  public abstract SerializedException getCause();
-}

+ 1 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/YarnServerBuilderUtils.java

@@ -22,12 +22,12 @@ import java.util.List;
 
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.records.MasterKey;
 import org.apache.hadoop.yarn.server.api.records.NodeAction;
-import org.apache.hadoop.yarn.server.api.records.SerializedException;
 import org.apache.hadoop.yarn.util.Records;
 
 /**
@@ -59,10 +59,4 @@ public class YarnServerBuilderUtils {
     }
     return response;
   }
-  
-  public static SerializedException newSerializedException(Throwable e) {
-    SerializedException se = Records.newRecord(SerializedException.class);
-    se.init(e);
-    return se;
-  }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/LocalResourceStatus.java

@@ -18,8 +18,8 @@
 package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords;
 
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.URL;
-import org.apache.hadoop.yarn.server.api.records.SerializedException;
 
 public interface LocalResourceStatus {
   public LocalResource getResource();

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/LocalResourceStatusPBImpl.java

@@ -18,9 +18,11 @@
 package org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.impl.pb;
 
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.ProtoBase;
+import org.apache.hadoop.yarn.api.records.impl.pb.SerializedExceptionPBImpl;
 import org.apache.hadoop.yarn.api.records.impl.pb.URLPBImpl;
 import org.apache.hadoop.yarn.proto.YarnProtos.LocalResourceProto;
 import org.apache.hadoop.yarn.proto.YarnProtos.SerializedExceptionProto;
@@ -28,8 +30,6 @@ import org.apache.hadoop.yarn.proto.YarnProtos.URLProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalResourceStatusProto;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalResourceStatusProtoOrBuilder;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.ResourceStatusTypeProto;
-import org.apache.hadoop.yarn.server.api.records.SerializedException;
-import org.apache.hadoop.yarn.server.api.records.impl.pb.SerializedExceptionPBImpl;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
 

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

@@ -23,12 +23,12 @@ import static org.apache.hadoop.service.Service.STATE.STARTED;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.SortedSet;
-import java.util.TreeMap;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.logging.Log;
@@ -50,25 +50,25 @@ import org.apache.hadoop.service.Service;
 import org.apache.hadoop.service.ServiceStateChangeListener;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.ContainerStatus;
 import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
 import org.apache.hadoop.yarn.exceptions.NMNotYetReadyException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.RPCUtil;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
@@ -130,8 +130,6 @@ public class ContainerManagerImpl extends CompositeService implements
 
   private final NodeStatusUpdater nodeStatusUpdater;
 
-  private final RecordFactory recordFactory = RecordFactoryProvider.getRecordFactory(null);
-
   protected LocalDirsHandlerService dirsHandler;
   protected final AsyncDispatcher dispatcher;
   private final ApplicationACLsManager aclsManager;
@@ -316,18 +314,25 @@ public class ContainerManagerImpl extends CompositeService implements
     return resultId;
   }
 
+  protected void authorizeUser(UserGroupInformation remoteUgi,
+      NMTokenIdentifier nmTokenIdentifier) throws YarnException {
+    if (!remoteUgi.getUserName().equals(
+      nmTokenIdentifier.getApplicationAttemptId().toString())) {
+      throw RPCUtil.getRemoteException("Expected applicationAttemptId: "
+          + remoteUgi.getUserName() + "Found: "
+          + nmTokenIdentifier.getApplicationAttemptId());
+    }
+  }
+
   /**
    * @param containerTokenIdentifier
    *          of the container to be started
-   * @param ugi
-   *          ugi corresponding to the remote end making the api-call
    * @throws YarnException
    */
   @Private
   @VisibleForTesting
   protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
-      ContainerTokenIdentifier containerTokenIdentifier,
-      UserGroupInformation ugi) throws YarnException {
+      ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
 
     ContainerId containerId = containerTokenIdentifier.getContainerID();
     String containerIDStr = containerId.toString();
@@ -342,12 +347,6 @@ public class ContainerManagerImpl extends CompositeService implements
         .append(" was used for starting container with container token")
         .append(" issued for application attempt : ")
         .append(containerId.getApplicationAttemptId());
-    } else if (!ugi.getUserName().equals(
-        nmTokenIdentifier.getApplicationAttemptId().toString())) {
-      unauthorized = true;
-      messageBuilder.append("\nExpected applicationAttemptId: ")
-        .append(ugi.getUserName()).append(" Found: ")
-        .append(nmTokenIdentifier.getApplicationAttemptId().toString());
     } else if (!this.context.getContainerTokenSecretManager()
         .isValidStartContainerRequest(containerTokenIdentifier)) {
       // Is the container being relaunched? Or RPC layer let startCall with
@@ -363,7 +362,6 @@ public class ContainerManagerImpl extends CompositeService implements
         .append(System.currentTimeMillis()).append(" found ")
         .append(containerTokenIdentifier.getExpiryTimeStamp());
     }
-
     if (unauthorized) {
       String msg = messageBuilder.toString();
       LOG.error(msg);
@@ -372,18 +370,53 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   /**
-   * Start a container on this NodeManager.
+   * Start a list of containers on this NodeManager.
    */
-  @SuppressWarnings("unchecked")
   @Override
-  public StartContainerResponse startContainer(StartContainerRequest request)
-      throws YarnException, IOException {
-
+  public StartContainersResponse
+      startContainers(StartContainersRequest requests) throws YarnException,
+          IOException {
     if (blockNewContainerRequests.get()) {
       throw new NMNotYetReadyException(
         "Rejecting new containers as NodeManager has not"
             + " yet connected with ResourceManager");
     }
+    UserGroupInformation remoteUgi = getRemoteUgi();
+    NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
+    authorizeUser(remoteUgi,nmTokenIdentifier);
+    List<ContainerId> succeededContainers = new ArrayList<ContainerId>();
+    Map<ContainerId, SerializedException> failedContainers =
+        new HashMap<ContainerId, SerializedException>();
+    for (StartContainerRequest request : requests.getStartContainerRequests()) {
+      ContainerId containerId = null;
+      try {
+        ContainerTokenIdentifier containerTokenIdentifier =
+            BuilderUtils.newContainerTokenIdentifier(request.getContainerToken());
+        verifyAndGetContainerTokenIdentifier(request.getContainerToken(),
+          containerTokenIdentifier);
+        containerId = containerTokenIdentifier.getContainerID();
+        startContainerInternal(nmTokenIdentifier, containerTokenIdentifier,
+          request);
+        succeededContainers.add(containerId);
+      } catch (YarnException e) {
+        failedContainers.put(containerId, SerializedException.newInstance(e));
+      } catch (InvalidToken ie) {
+        failedContainers.put(containerId, SerializedException.newInstance(ie));
+        throw ie;
+      } catch (IOException e) {
+        throw RPCUtil.getRemoteException(e);
+      }
+    }
+
+    return StartContainersResponse.newInstance(auxiliaryServices.getMetaData(),
+      succeededContainers, failedContainers);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void startContainerInternal(NMTokenIdentifier nmTokenIdentifier,
+      ContainerTokenIdentifier containerTokenIdentifier,
+      StartContainerRequest request) throws YarnException, IOException {
+
     /*
      * 1) It should save the NMToken into NMTokenSecretManager. This is done
      * here instead of RPC layer because at the time of opening/authenticating
@@ -395,18 +428,8 @@ public class ContainerManagerImpl extends CompositeService implements
      * belongs to correct Node Manager (part of retrieve password). c) It has
      * correct RMIdentifier. d) It is not expired.
      */
-    // update NMToken
-
-    UserGroupInformation remoteUgi = getRemoteUgi();
-    NMTokenIdentifier nmTokenIdentifier = selectNMTokenIdentifier(remoteUgi);
-    
-    // Validate containerToken
-    ContainerTokenIdentifier containerTokenIdentifier =
-        verifyAndGetContainerTokenIdentifier(request.getContainerToken());
-
-    authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier,
-      remoteUgi);
-
+    authorizeStartRequest(nmTokenIdentifier, containerTokenIdentifier);
+ 
     if (containerTokenIdentifier.getRMIdentifer() != nodeStatusUpdater
         .getRMIdentifier()) {
         // Is the container coming from unknown RM
@@ -415,9 +438,9 @@ public class ContainerManagerImpl extends CompositeService implements
           .append(" rejected as it is allocated by a previous RM");
         throw new InvalidContainerException(sb.toString());
     }
-    
+    // update NMToken
     updateNMTokenIdentifier(nmTokenIdentifier);
-    
+
     ContainerId containerId = containerTokenIdentifier.getContainerID();
     String containerIdStr = containerId.toString();
     String user = containerTokenIdentifier.getApplicationSubmitter();
@@ -461,26 +484,16 @@ public class ContainerManagerImpl extends CompositeService implements
       containerTokenIdentifier);
     NMAuditLogger.logSuccess(user, AuditConstants.START_CONTAINER,
       "ContainerManageImpl", applicationID, containerId);
-    StartContainerResponse response =
-        recordFactory.newRecordInstance(StartContainerResponse.class);
-    response.setAllServicesMetaData(auxiliaryServices.getMetaData());
     // TODO launchedContainer misplaced -> doesn't necessarily mean a container
     // launch. A finished Application will not launch containers.
     metrics.launchedContainer();
-    metrics.allocateContainer(containerTokenIdentifier.getResource());
-    return response;
+    metrics.allocateContainer(containerTokenIdentifier.getResource()); 
   }
 
   protected ContainerTokenIdentifier verifyAndGetContainerTokenIdentifier(
-      org.apache.hadoop.yarn.api.records.Token token) throws YarnException,
+      org.apache.hadoop.yarn.api.records.Token token,
+      ContainerTokenIdentifier containerTokenIdentifier) throws YarnException,
       InvalidToken {
-    ContainerTokenIdentifier containerTokenIdentifier = null;
-    try {
-      containerTokenIdentifier =
-          BuilderUtils.newContainerTokenIdentifier(token);
-    } catch (IOException e) {
-      throw RPCUtil.getRemoteException(e);
-    }
     byte[] password =
         context.getContainerTokenSecretManager().retrievePassword(
           containerTokenIdentifier);
@@ -528,21 +541,37 @@ public class ContainerManagerImpl extends CompositeService implements
   }
 
   /**
-   * Stop the container running on this NodeManager.
+   * Stop a list of containers running on this NodeManager.
    */
   @Override
-  @SuppressWarnings("unchecked")
-  public StopContainerResponse stopContainer(StopContainerRequest request)
+  public StopContainersResponse stopContainers(StopContainersRequest requests)
       throws YarnException, IOException {
 
-    ContainerId containerID = request.getContainerId();
+    List<ContainerId> succeededRequests = new ArrayList<ContainerId>();
+    Map<ContainerId, SerializedException> failedRequests =
+        new HashMap<ContainerId, SerializedException>();
+    UserGroupInformation remoteUgi = getRemoteUgi();
+    NMTokenIdentifier identifier = selectNMTokenIdentifier(remoteUgi);
+    for (ContainerId id : requests.getContainerIds()) {
+      try {
+        stopContainerInternal(identifier, id);
+        succeededRequests.add(id);
+      } catch (YarnException e) {
+        failedRequests.put(id, SerializedException.newInstance(e));
+      }
+    }
+    return StopContainersResponse
+      .newInstance(succeededRequests, failedRequests);
+  }
+
+  @SuppressWarnings("unchecked")
+  private void stopContainerInternal(NMTokenIdentifier nmTokenIdentifier,
+      ContainerId containerID) throws YarnException {
     String containerIDStr = containerID.toString();
     Container container = this.context.getContainers().get(containerID);
-    LOG.info("Getting container-status for " + containerIDStr);
-    authorizeGetAndStopContainerRequest(containerID, container, true);
-
-    StopContainerResponse response =
-        recordFactory.newRecordInstance(StopContainerResponse.class);
+    LOG.info("Stopping container with container Id: " + containerIDStr);
+    authorizeGetAndStopContainerRequest(containerID, container, true,
+      nmTokenIdentifier);
 
     dispatcher.getEventHandler().handle(
       new ContainerKillEvent(containerID,
@@ -555,37 +584,51 @@ public class ContainerManagerImpl extends CompositeService implements
     // TODO: Move this code to appropriate place once kill_container is
     // implemented.
     nodeStatusUpdater.sendOutofBandHeartBeat();
-
-    return response;
   }
 
+  /**
+   * Get a list of container statuses running on this NodeManager
+   */
   @Override
-  public GetContainerStatusResponse getContainerStatus(
-      GetContainerStatusRequest request) throws YarnException, IOException {
+  public GetContainerStatusesResponse getContainerStatuses(
+      GetContainerStatusesRequest request) throws YarnException, IOException {
+
+    List<ContainerStatus> succeededRequests = new ArrayList<ContainerStatus>();
+    Map<ContainerId, SerializedException> failedRequests =
+        new HashMap<ContainerId, SerializedException>();
+    UserGroupInformation remoteUgi = getRemoteUgi();
+    NMTokenIdentifier identifier = selectNMTokenIdentifier(remoteUgi);
+    for (ContainerId id : request.getContainerIds()) {
+      try {
+        ContainerStatus status = getContainerStatusInternal(id, identifier);
+        succeededRequests.add(status);
+      } catch (YarnException e) {
+        failedRequests.put(id, SerializedException.newInstance(e));
+      }
+    }
+    return GetContainerStatusesResponse.newInstance(succeededRequests,
+      failedRequests);
+  }
 
-    ContainerId containerID = request.getContainerId();
+  private ContainerStatus getContainerStatusInternal(ContainerId containerID,
+      NMTokenIdentifier nmTokenIdentifier) throws YarnException {
     String containerIDStr = containerID.toString();
     Container container = this.context.getContainers().get(containerID);
 
     LOG.info("Getting container-status for " + containerIDStr);
-    authorizeGetAndStopContainerRequest(containerID, container, false);
+    authorizeGetAndStopContainerRequest(containerID, container, false,
+      nmTokenIdentifier);
 
     ContainerStatus containerStatus = container.cloneAndGetContainerStatus();
     LOG.info("Returning " + containerStatus);
-    GetContainerStatusResponse response =
-        recordFactory.newRecordInstance(GetContainerStatusResponse.class);
-    response.setStatus(containerStatus);
-    return response;
+    return containerStatus;
   }
 
   @Private
   @VisibleForTesting
   protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
-      Container container, boolean stopRequest) throws YarnException {
-
-    UserGroupInformation remoteUgi = getRemoteUgi();
-    NMTokenIdentifier identifier = selectNMTokenIdentifier(remoteUgi);
-
+      Container container, boolean stopRequest, NMTokenIdentifier identifier)
+      throws YarnException {
     /*
      * For get/stop container status; we need to verify that 1) User (NMToken)
      * application attempt only has started container. 2) Requested containerId

+ 1 - 1
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

@@ -103,7 +103,7 @@ public class ContainerImpl implements Container {
   public ContainerImpl(Configuration conf, Dispatcher dispatcher,
       ContainerLaunchContext launchContext, Credentials creds,
       NodeManagerMetrics metrics,
-      ContainerTokenIdentifier containerTokenIdentifier) throws IOException {
+      ContainerTokenIdentifier containerTokenIdentifier) {
     this.daemonConf = conf;
     this.dispatcher = dispatcher;
     this.launchContext = launchContext;

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/ContainerLocalizer.java

@@ -53,6 +53,7 @@ import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.util.DiskChecker;
 import org.apache.hadoop.yarn.YarnUncaughtExceptionHandler;
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
@@ -288,11 +289,10 @@ public class ContainerLocalizer {
           stat.setStatus(ResourceStatusType.FETCH_SUCCESS);
         } catch (ExecutionException e) {
           stat.setStatus(ResourceStatusType.FETCH_FAILURE);
-          stat.setException(
-              YarnServerBuilderUtils.newSerializedException(e.getCause()));
+          stat.setException(SerializedException.newInstance(e.getCause()));
         } catch (CancellationException e) {
           stat.setStatus(ResourceStatusType.FETCH_FAILURE);
-          stat.setException(YarnServerBuilderUtils.newSerializedException(e));
+          stat.setException(SerializedException.newInstance(e));
         }
         // TODO shouldn't remove until ACK
         i.remove();

+ 2 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/DummyContainerManager.java

@@ -195,14 +195,13 @@ public class DummyContainerManager extends ContainerManagerImpl {
   
   @Override
   protected void authorizeStartRequest(NMTokenIdentifier nmTokenIdentifier,
-      ContainerTokenIdentifier containerTokenIdentifier,
-      UserGroupInformation ugi) throws YarnException {
+      ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
     // do nothing
   }
   
   @Override
   protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
-      Container container, boolean stopRequest) throws YarnException {
+      Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException {
     // do nothing
   }
 

+ 24 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestContainerManagerWithLCE.java

@@ -146,7 +146,7 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
 
   @Override
   public void testContainerLaunchFromPreviousRM() throws InterruptedException,
-      IOException {
+      IOException, YarnException {
     // Don't run the test if the binary is not available.
     if (!shouldRunTest()) {
       LOG.info("LCE binary path is not passed. Not running the test");
@@ -155,6 +155,29 @@ public class TestContainerManagerWithLCE extends TestContainerManager {
     LOG.info("Running testContainerLaunchFromPreviousRM");
     super.testContainerLaunchFromPreviousRM();
   }
+
+  @Override
+  public void testMultipleContainersLaunch() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerLaunchFromPreviousRM");
+    super.testMultipleContainersLaunch();
+  }
+
+  @Override
+  public void testMultipleContainersStopAndGetStatus() throws Exception {
+    // Don't run the test if the binary is not available.
+    if (!shouldRunTest()) {
+      LOG.info("LCE binary path is not passed. Not running the test");
+      return;
+    }
+    LOG.info("Running testContainerLaunchFromPreviousRM");
+    super.testMultipleContainersStopAndGetStatus();
+  }
+
   private boolean shouldRunTest() {
     return System
         .getProperty(YarnConfiguration.NM_LINUX_CONTAINER_EXECUTOR_PATH) != null;

+ 20 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java

@@ -20,18 +20,19 @@ package org.apache.hadoop.yarn.server.nodemanager;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 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.Token;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -46,7 +47,6 @@ 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.NMTokenSecretManagerInNM;
 import org.apache.hadoop.yarn.server.security.ApplicationACLsManager;
-import org.apache.hadoop.yarn.server.utils.BuilderUtils;
 import org.junit.Test;
 
 
@@ -62,7 +62,7 @@ public class TestEventFlow {
   private static File remoteLogDir = new File("target",
       TestEventFlow.class.getName() + "-remoteLogDir").getAbsoluteFile();
   private static final long SIMULATED_RM_IDENTIFIER = 1234;
-
+  
   @Test
   public void testSuccessfulContainerLaunch() throws InterruptedException,
       IOException, YarnException {
@@ -140,21 +140,25 @@ public class TestEventFlow {
     ContainerId cID = ContainerId.newInstance(applicationAttemptId, 0);
 
     String user = "testing";
-    StartContainerRequest request = 
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    request.setContainerLaunchContext(launchContext);
-    request.setContainerToken(TestContainerManager.createContainerToken(cID,
-      SIMULATED_RM_IDENTIFIER, context.getNodeId(), user,
-      context.getContainerTokenSecretManager()));
-    containerManager.startContainer(request);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(launchContext,
+          TestContainerManager.createContainerToken(cID,
+            SIMULATED_RM_IDENTIFIER, context.getNodeId(), user,
+            context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
         ContainerState.RUNNING);
 
-    StopContainerRequest stopRequest = 
-        recordFactory.newRecordInstance(StopContainerRequest.class);
-    stopRequest.setContainerId(cID);
-    containerManager.stopContainer(stopRequest);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cID);
+    StopContainersRequest stopRequest =
+        StopContainersRequest.newInstance(containerIds);
+    containerManager.stopContainers(stopRequest);
     BaseContainerManagerTest.waitForContainerState(containerManager, cID,
         ContainerState.COMPLETE);
 

+ 18 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java

@@ -39,8 +39,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -129,13 +130,17 @@ public class TestNodeManagerReboot {
     List<String> commands = new ArrayList<String>();
     containerLaunchContext.setCommands(commands);
 
-    final StartContainerRequest startRequest =
-        Records.newRecord(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
     NodeId nodeId = nm.getNMContext().getNodeId();
-    startRequest.setContainerToken(TestContainerManager.createContainerToken(
-      cId, 0, nodeId, destinationFile, nm.getNMContext()
-        .getContainerTokenSecretManager()));
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          TestContainerManager.createContainerToken(
+            cId, 0, nodeId, destinationFile, nm.getNMContext()
+              .getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    final StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+
     final UserGroupInformation currentUser =
         UserGroupInformation.createRemoteUser(cId.getApplicationAttemptId()
           .toString());
@@ -145,16 +150,17 @@ public class TestNodeManagerReboot {
     currentUser.doAs(new PrivilegedExceptionAction<Void>() {
       @Override
       public Void run() throws YarnException, IOException {
-        nm.getContainerManager().startContainer(startRequest);
+        nm.getContainerManager().startContainers(allRequests);
         return null;
       }
     });
 
-    GetContainerStatusRequest request =
-        Records.newRecord(GetContainerStatusRequest.class);
-    request.setContainerId(cId);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cId);
+    GetContainerStatusesRequest request =
+        GetContainerStatusesRequest.newInstance(containerIds);
     Container container =
-        nm.getNMContext().getContainers().get(request.getContainerId());
+        nm.getNMContext().getContainers().get(request.getContainerIds().get(0));
 
     final int MAX_TRIES = 20;
     int numTries = 0;

+ 11 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.concurrent.BrokenBarrierException;
 import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CyclicBarrier;
@@ -31,6 +33,7 @@ import org.apache.hadoop.fs.FileContext;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -282,15 +285,18 @@ public class TestNodeManagerResync {
         try {
           while (!isStopped && numContainers < 10) {
             ContainerId cId = TestNodeManagerShutdown.createContainerId();
-            StartContainerRequest startRequest =
-                recordFactory.newRecordInstance(StartContainerRequest.class);
-            startRequest.setContainerLaunchContext(containerLaunchContext);
-            startRequest.setContainerToken(null);
+            StartContainerRequest scRequest =
+                StartContainerRequest.newInstance(containerLaunchContext,
+                  null);
+            List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+            list.add(scRequest);
+            StartContainersRequest allRequests =
+                StartContainersRequest.newInstance(list);
             System.out.println("no. of containers to be launched: "
                 + numContainers);
             numContainers++;
             try {
-              getContainerManager().startContainer(startRequest);
+              getContainerManager().startContainers(allRequests);
             } catch (YarnException e) {
               numContainersRejected++;
               Assert.assertTrue(e.getMessage().contains(

+ 17 - 13
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java

@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -40,8 +41,9 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -53,7 +55,6 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.URL;
-import org.apache.hadoop.yarn.api.records.impl.pb.ProtoUtils;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.Dispatcher;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -181,12 +182,6 @@ public class TestNodeManagerShutdown {
     containerLaunchContext.setLocalResources(localResources);
     List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
     containerLaunchContext.setCommands(commands);
-    StartContainerRequest startRequest =
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
-    startRequest
-      .setContainerToken(TestContainerManager.createContainerToken(cId, 0,
-        nodeId, user, nm.getNMContext().getContainerTokenSecretManager()));
     final InetSocketAddress containerManagerBindAddress =
         NetUtils.createSocketAddrForHost("127.0.0.1", 12345);
     UserGroupInformation currentUser = UserGroupInformation
@@ -210,13 +205,22 @@ public class TestNodeManagerShutdown {
               containerManagerBindAddress, conf);
           }
         });
-    containerManager.startContainer(startRequest);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          TestContainerManager.createContainerToken(cId, 0,
+            nodeId, user, nm.getNMContext().getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
     
-    GetContainerStatusRequest request =
-        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-        request.setContainerId(cId);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cId);
+    GetContainerStatusesRequest request =
+        GetContainerStatusesRequest.newInstance(containerIds);
     ContainerStatus containerStatus =
-        containerManager.getContainerStatus(request).getStatus();
+        containerManager.getContainerStatuses(request).getContainerStatuses().get(0);
     Assert.assertEquals(ContainerState.RUNNING, containerStatus.getState());
   }
   

+ 1 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java

@@ -45,8 +45,8 @@ import org.apache.hadoop.io.retry.RetryPolicy;
 import org.apache.hadoop.io.retry.RetryProxy;
 import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
 import org.apache.hadoop.net.NetUtils;
-import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.service.Service.STATE;
+import org.apache.hadoop.service.ServiceOperations;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -65,7 +65,6 @@ import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.security.ContainerTokenIdentifier;
 import org.apache.hadoop.yarn.server.api.ResourceTracker;
-import org.apache.hadoop.yarn.server.api.ServerRMProxy;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatRequest;
 import org.apache.hadoop.yarn.server.api.protocolrecords.NodeHeartbeatResponse;
 import org.apache.hadoop.yarn.server.api.protocolrecords.RegisterNodeManagerRequest;

+ 35 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/api/protocolrecords/impl/pb/TestPBRecordImpl.java

@@ -21,16 +21,21 @@ import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
+import java.io.IOException;
 import java.net.URISyntaxException;
 import java.util.ArrayList;
 
+import junit.framework.Assert;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.DataInputBuffer;
 import org.apache.hadoop.io.DataOutputBuffer;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.impl.pb.LocalResourcePBImpl;
+import org.apache.hadoop.yarn.exceptions.YarnException;
 import org.apache.hadoop.yarn.factories.RecordFactory;
 import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.proto.YarnServerNodemanagerServiceProtos.LocalResourceStatusProto;
@@ -42,7 +47,6 @@ import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerAc
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerHeartbeatResponse;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalizerStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
-import org.apache.hadoop.yarn.server.utils.YarnServerBuilderUtils;
 import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.junit.Test;
 
@@ -80,7 +84,7 @@ public class TestPBRecordImpl {
     e.setStackTrace(new StackTraceElement[] {
         new StackTraceElement("foo", "bar", "baz", 10),
         new StackTraceElement("sbb", "one", "onm", 10) });
-    ret.setException(YarnServerBuilderUtils.newSerializedException(e));
+    ret.setException(SerializedException.newInstance(e));
     return ret;
   }
 
@@ -176,4 +180,33 @@ public class TestPBRecordImpl {
     assertEquals(createResource(), rsrcD.getResourceSpecs().get(0).getResource());
   }
 
+
+  @Test(timeout=10000)
+  public void testSerializedExceptionDeSer() throws Exception{
+    // without cause
+    YarnException yarnEx = new YarnException("Yarn_Exception");
+    SerializedException serEx = SerializedException.newInstance(yarnEx);
+    Throwable throwable = serEx.deSerialize();
+    Assert.assertEquals(yarnEx.getClass(), throwable.getClass());
+    Assert.assertEquals(yarnEx.getMessage(), throwable.getMessage());
+
+    // with cause
+    IOException ioe = new IOException("Test_IOException");
+    RuntimeException runtimeException =
+        new RuntimeException("Test_RuntimeException", ioe);
+    YarnException yarnEx2 =
+        new YarnException("Test_YarnException", runtimeException);
+
+    SerializedException serEx2 = SerializedException.newInstance(yarnEx2);
+    Throwable throwable2 = serEx2.deSerialize();
+    throwable2.printStackTrace();
+    Assert.assertEquals(yarnEx2.getClass(), throwable2.getClass());
+    Assert.assertEquals(yarnEx2.getMessage(), throwable2.getMessage());
+
+    Assert.assertEquals(runtimeException.getClass(), throwable2.getCause().getClass());
+    Assert.assertEquals(runtimeException.getMessage(), throwable2.getCause().getMessage());
+
+    Assert.assertEquals(ioe.getClass(), throwable2.getCause().getCause().getClass());
+    Assert.assertEquals(ioe.getMessage(), throwable2.getCause().getCause().getMessage());
+  }
 }

+ 19 - 12
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/BaseContainerManagerTest.java

@@ -20,6 +20,8 @@ package org.apache.hadoop.yarn.server.nodemanager.containermanager;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
 
 import junit.framework.Assert;
 
@@ -32,7 +34,7 @@ import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
@@ -189,15 +191,18 @@ public abstract class BaseContainerManagerTest {
 
       @Override
         protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
-            Container container, boolean stopRequest) throws YarnException {
+            Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException {
           // do nothing
         }
-      
+      @Override
+      protected void authorizeUser(UserGroupInformation remoteUgi,
+          NMTokenIdentifier nmTokenIdentifier) {
+        // do nothing
+      }
       @Override
         protected void authorizeStartRequest(
             NMTokenIdentifier nmTokenIdentifier,
-            ContainerTokenIdentifier containerTokenIdentifier,
-            UserGroupInformation ugi) throws YarnException {
+            ContainerTokenIdentifier containerTokenIdentifier) throws YarnException {
           // do nothing
         }
       
@@ -238,18 +243,20 @@ public abstract class BaseContainerManagerTest {
   public static void waitForContainerState(ContainerManagementProtocol containerManager,
           ContainerId containerID, ContainerState finalState, int timeOutMax)
           throws InterruptedException, YarnException, IOException {
-    GetContainerStatusRequest request =
-        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-        request.setContainerId(containerID);
-        ContainerStatus containerStatus =
-            containerManager.getContainerStatus(request).getStatus();
-        int timeoutSecs = 0;
+    List<ContainerId> list = new ArrayList<ContainerId>();
+    list.add(containerID);
+    GetContainerStatusesRequest request =
+        GetContainerStatusesRequest.newInstance(list);
+    ContainerStatus containerStatus =
+        containerManager.getContainerStatuses(request).getContainerStatuses()
+          .get(0);
+    int timeoutSecs = 0;
       while (!containerStatus.getState().equals(finalState)
           && timeoutSecs++ < timeOutMax) {
           Thread.sleep(1000);
           LOG.info("Waiting for container to get into state " + finalState
               + ". Current state is " + containerStatus.getState());
-          containerStatus = containerManager.getContainerStatus(request).getStatus();
+          containerStatus = containerManager.getContainerStatuses(request).getContainerStatuses().get(0);
         }
         LOG.info("Container state is " + containerStatus.getState());
         Assert.assertEquals("ContainerState is not correct (timedout)",

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

@@ -24,6 +24,7 @@ import java.io.FileReader;
 import java.io.IOException;
 import java.io.PrintWriter;
 import java.net.InetAddress;
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
@@ -37,9 +38,13 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.Shell;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -51,6 +56,7 @@ import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.exceptions.InvalidContainerException;
@@ -63,6 +69,7 @@ import org.apache.hadoop.yarn.server.nodemanager.ContainerExecutor.ExitCode;
 import org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor;
 import org.apache.hadoop.yarn.server.nodemanager.DeletionService;
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.application.ApplicationState;
+import org.apache.hadoop.yarn.server.nodemanager.containermanager.container.Container;
 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.security.NMContainerTokenSecretManager;
@@ -88,11 +95,11 @@ public class TestContainerManager extends BaseContainerManagerTest {
     super.setup();
   }
 
-  private ContainerId createContainerId() {
+  private ContainerId createContainerId(int id) {
     ApplicationId appId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId containerId = ContainerId.newInstance(appAttemptId, id);
     return containerId;
   }
   
@@ -119,6 +126,14 @@ public class TestContainerManager extends BaseContainerManagerTest {
           .getKeyId()));
         return ugi;
       }
+
+      @Override
+      protected void authorizeGetAndStopContainerRequest(ContainerId containerId,
+          Container container, boolean stopRequest, NMTokenIdentifier identifier) throws YarnException {
+        if(container == null || container.getUser().equals("Fail")){
+          throw new YarnException("Reject this container");
+        }
+      }
     };
   }
   
@@ -138,12 +153,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
     // Just do a query for a non-existing container.
     boolean throwsException = false;
     try {
-      GetContainerStatusRequest request = 
-          recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-      ContainerId cId = createContainerId();
-      request.setContainerId(cId);
-      containerManager.getContainerStatus(request);
-    } catch (YarnException e) {
+      List<ContainerId> containerIds = new ArrayList<ContainerId>();
+      ContainerId id =createContainerId(0);
+      containerIds.add(id);
+      GetContainerStatusesRequest request =
+          GetContainerStatusesRequest.newInstance(containerIds);
+      GetContainerStatusesResponse response =
+          containerManager.getContainerStatuses(request);
+      if(response.getFailedRequests().containsKey(id)){
+        throw response.getFailedRequests().get(id).deSerialize();
+      }
+    } catch (Throwable e) {
       throwsException = true;
     }
     Assert.assertTrue(throwsException);
@@ -163,7 +183,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     fileWriter.close();
 
     // ////// Construct the Container-id
-    ContainerId cId = createContainerId();
+    ContainerId cId = createContainerId(0);
 
     // ////// Construct the container-spec.
     ContainerLaunchContext containerLaunchContext = 
@@ -182,14 +202,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
         new HashMap<String, LocalResource>();
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);
-    StartContainerRequest startRequest = 
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
-    startRequest.setContainerToken(createContainerToken(cId,
-      DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
-      context.getContainerTokenSecretManager()));
 
-    containerManager.startContainer(startRequest);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+          containerLaunchContext,
+          createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
         ContainerState.COMPLETE);
@@ -237,7 +260,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
         new File(tmpDir, "start_file.txt").getAbsoluteFile();
 
     // ////// Construct the Container-id
-    ContainerId cId = createContainerId();
+    ContainerId cId = createContainerId(0);
 
     if (Shell.WINDOWS) {
       fileWriter.println("@echo Hello World!> " + processStartFile);
@@ -272,13 +295,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
     List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
     containerLaunchContext.setCommands(commands);
 
-    StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
-    startRequest.setContainerToken(createContainerToken(cId,
-      DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
-      context.getContainerTokenSecretManager()));
-    containerManager.startContainer(startRequest);
- 
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          createContainerToken(cId,
+            DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
+            context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
+
     int timeoutSecs = 0;
     while (!processStartFile.exists() && timeoutSecs++ < 20) {
       Thread.sleep(1000);
@@ -305,18 +332,18 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertTrue("Process is not alive!",
       DefaultContainerExecutor.containerIsAlive(pid));
 
-    StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
-    stopRequest.setContainerId(cId);
-    containerManager.stopContainer(stopRequest);
-
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cId);
+    StopContainersRequest stopRequest =
+        StopContainersRequest.newInstance(containerIds);
+    containerManager.stopContainers(stopRequest);
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
         ContainerState.COMPLETE);
     
-    GetContainerStatusRequest gcsRequest = 
-        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-    gcsRequest.setContainerId(cId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
     ContainerStatus containerStatus = 
-        containerManager.getContainerStatus(gcsRequest).getStatus();
+        containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
     int expectedExitCode = Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
       ExitCode.TERMINATED.getExitCode();
     Assert.assertEquals(expectedExitCode, containerStatus.getExitStatus());
@@ -325,7 +352,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertFalse("Process is still alive!",
       DefaultContainerExecutor.containerIsAlive(pid));
   }
-  
+
   private void testContainerLaunchAndExit(int exitCode) throws IOException,
       InterruptedException, YarnException {
 
@@ -335,7 +362,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
 			  new File(tmpDir, "start_file.txt").getAbsoluteFile();
 
 	  // ////// Construct the Container-id
-	  ContainerId cId = createContainerId();
+	  ContainerId cId = createContainerId(0);
 
 	  if (Shell.WINDOWS) {
 	    fileWriter.println("@echo Hello World!> " + processStartFile);
@@ -376,21 +403,26 @@ public class TestContainerManager extends BaseContainerManagerTest {
 	  List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
 	  containerLaunchContext.setCommands(commands);
 
-	  StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
-	  startRequest.setContainerLaunchContext(containerLaunchContext);
-    startRequest.setContainerToken(createContainerToken(cId,
-      DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
-      context.getContainerTokenSecretManager()));
-	  containerManager.startContainer(startRequest);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+          containerLaunchContext,
+          createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
 
 	  BaseContainerManagerTest.waitForContainerState(containerManager, cId,
 			  ContainerState.COMPLETE);
 
-	  GetContainerStatusRequest gcsRequest = 
-			  recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-	  gcsRequest.setContainerId(cId);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
 	  ContainerStatus containerStatus = 
-			  containerManager.getContainerStatus(gcsRequest).getStatus();
+			  containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
 
 	  // Verify exit status matches exit state of script
 	  Assert.assertEquals(exitCode,
@@ -439,7 +471,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     fileWriter.close();
 
     // ////// Construct the Container-id
-    ContainerId cId = createContainerId();
+    ContainerId cId = createContainerId(0);
     ApplicationId appId = cId.getApplicationAttemptId().getApplicationId();
 
     // ////// Construct the container-spec.
@@ -460,11 +492,17 @@ public class TestContainerManager extends BaseContainerManagerTest {
         new HashMap<String, LocalResource>();
     localResources.put(destinationFile, rsrc_alpha);
     containerLaunchContext.setLocalResources(localResources);
-    StartContainerRequest request = recordFactory.newRecordInstance(StartContainerRequest.class);
-    request.setContainerLaunchContext(containerLaunchContext);
-    request.setContainerToken(createContainerToken(cId, DUMMY_RM_IDENTIFIER,
-      context.getNodeId(), user, context.getContainerTokenSecretManager()));
-    containerManager.startContainer(request);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(
+          containerLaunchContext,
+          createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
         ContainerState.COMPLETE);
@@ -528,29 +566,37 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
   @Test
   public void testContainerLaunchFromPreviousRM() throws IOException,
-      InterruptedException {
+      InterruptedException, YarnException {
     containerManager.start();
 
     ContainerLaunchContext containerLaunchContext =
         recordFactory.newRecordInstance(ContainerLaunchContext.class);
 
-    ContainerId cId1 = createContainerId();
-    ContainerId cId2 = createContainerId();
+    ContainerId cId1 = createContainerId(0);
+    ContainerId cId2 = createContainerId(0);
     containerLaunchContext
       .setLocalResources(new HashMap<String, LocalResource>());
 
     // Construct the Container with Invalid RMIdentifier
     StartContainerRequest startRequest1 =
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest1.setContainerLaunchContext(containerLaunchContext);
+        StartContainerRequest.newInstance(containerLaunchContext,
+          createContainerToken(cId1,
+            ResourceManagerConstants.RM_INVALID_IDENTIFIER, context.getNodeId(),
+            user, context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(startRequest1);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
     
-    startRequest1.setContainerToken(createContainerToken(cId1,
-      ResourceManagerConstants.RM_INVALID_IDENTIFIER, context.getNodeId(),
-      user, context.getContainerTokenSecretManager()));
     boolean catchException = false;
     try {
-      containerManager.startContainer(startRequest1);
-    } catch (YarnException e) {
+      StartContainersResponse response = containerManager.startContainers(allRequests);
+      if(response.getFailedRequests().containsKey(cId1)) {
+        throw response.getFailedRequests().get(cId1).deSerialize();
+      }
+    } catch (Throwable e) {
+      e.printStackTrace();
       catchException = true;
       Assert.assertTrue(e.getMessage().contains(
         "Container " + cId1 + " rejected as it is allocated by a previous RM"));
@@ -563,21 +609,143 @@ public class TestContainerManager extends BaseContainerManagerTest {
 
     // Construct the Container with a RMIdentifier within current RM
     StartContainerRequest startRequest2 =
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest2.setContainerLaunchContext(containerLaunchContext);
-    startRequest2.setContainerToken(createContainerToken(cId2,
-      DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
-      context.getContainerTokenSecretManager()));
+        StartContainerRequest.newInstance(containerLaunchContext,
+          createContainerToken(cId2,
+            DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
+            context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list2 = new ArrayList<StartContainerRequest>();
+    list.add(startRequest2);
+    StartContainersRequest allRequests2 =
+        StartContainersRequest.newInstance(list2);
+    containerManager.startContainers(allRequests2);
+    
     boolean noException = true;
     try {
-      containerManager.startContainer(startRequest2);
+      containerManager.startContainers(allRequests2);
     } catch (YarnException e) {
       noException = false;
     }
     // Verify that startContainer get no YarnException
     Assert.assertTrue(noException);
   }
-  
+
+  @Test
+  public void testMultipleContainersLaunch() throws Exception {
+    containerManager.start();
+
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+    for (int i = 0; i < 10; i++) {
+      ContainerId cId = createContainerId(i);
+      long identifier = 0;
+      if ((i & 1) == 0)
+        // container with even id fail
+        identifier = ResourceManagerConstants.RM_INVALID_IDENTIFIER;
+      else
+        identifier = DUMMY_RM_IDENTIFIER;
+      Token containerToken =
+          createContainerToken(cId, identifier, context.getNodeId(), user,
+            context.getContainerTokenSecretManager());
+      StartContainerRequest request =
+          StartContainerRequest.newInstance(containerLaunchContext,
+            containerToken);
+      list.add(request);
+    }
+    StartContainersRequest requestList =
+        StartContainersRequest.newInstance(list);
+
+    StartContainersResponse response =
+        containerManager.startContainers(requestList);
+
+    Assert.assertEquals(5, response.getSuccessfullyStartedContainers().size());
+    for (ContainerId id : response.getSuccessfullyStartedContainers()) {
+      // Containers with odd id should succeed.
+      Assert.assertEquals(1, id.getId() & 1);
+    }
+    Assert.assertEquals(5, response.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : response
+      .getFailedRequests().entrySet()) {
+      // Containers with even id should fail.
+      Assert.assertEquals(0, entry.getKey().getId() & 1);
+      Assert.assertTrue(entry.getValue().getMessage()
+        .contains(
+          "Container " + entry.getKey() + " rejected as it is allocated by a previous RM"));
+    }
+  }
+
+  @Test
+  public void testMultipleContainersStopAndGetStatus() throws Exception {
+    containerManager.start();
+    List<StartContainerRequest> startRequest =
+        new ArrayList<StartContainerRequest>();
+    ContainerLaunchContext containerLaunchContext =
+        recordFactory.newRecordInstance(ContainerLaunchContext.class);
+
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    for (int i = 0; i < 10; i++) {
+      ContainerId cId = createContainerId(i);
+      String user = null;
+      if ((i & 1) == 0) {
+        // container with even id fail
+        user = "Fail";
+      } else {
+        user = "Pass";
+      }
+      Token containerToken =
+          createContainerToken(cId, DUMMY_RM_IDENTIFIER, context.getNodeId(),
+            user, context.getContainerTokenSecretManager());
+      StartContainerRequest request =
+          StartContainerRequest.newInstance(containerLaunchContext,
+            containerToken);
+      startRequest.add(request);
+      containerIds.add(cId);
+    }
+    // start containers
+    StartContainersRequest requestList =
+        StartContainersRequest.newInstance(startRequest);
+    containerManager.startContainers(requestList);
+
+    // Get container statuses
+    GetContainerStatusesRequest statusRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    GetContainerStatusesResponse statusResponse =
+        containerManager.getContainerStatuses(statusRequest);
+    Assert.assertEquals(5, statusResponse.getContainerStatuses().size());
+    for (ContainerStatus status : statusResponse.getContainerStatuses()) {
+      // Containers with odd id should succeed
+      Assert.assertEquals(1, status.getContainerId().getId() & 1);
+    }
+    Assert.assertEquals(5, statusResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : statusResponse
+      .getFailedRequests().entrySet()) {
+      // Containers with even id should fail.
+      Assert.assertEquals(0, entry.getKey().getId() & 1);
+      Assert.assertTrue(entry.getValue().getMessage()
+        .contains("Reject this container"));
+    }
+
+    // stop containers
+    StopContainersRequest stopRequest =
+        StopContainersRequest.newInstance(containerIds);
+    StopContainersResponse stopResponse =
+        containerManager.stopContainers(stopRequest);
+    Assert.assertEquals(5, stopResponse.getSuccessfullyStoppedContainers()
+      .size());
+    for (ContainerId id : stopResponse.getSuccessfullyStoppedContainers()) {
+      // Containers with odd id should succeed.
+      Assert.assertEquals(1, id.getId() & 1);
+    }
+    Assert.assertEquals(5, stopResponse.getFailedRequests().size());
+    for (Map.Entry<ContainerId, SerializedException> entry : stopResponse
+      .getFailedRequests().entrySet()) {
+      // Containers with even id should fail.
+      Assert.assertEquals(0, entry.getKey().getId() & 1);
+      Assert.assertTrue(entry.getValue().getMessage()
+        .contains("Reject this container"));
+    }
+  }
+
   public static Token createContainerToken(ContainerId cId, long rmIdentifier,
       NodeId nodeId, String user,
       NMContainerTokenSecretManager containerTokenSecretManager)

+ 39 - 25
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

@@ -42,9 +42,10 @@ import org.apache.hadoop.util.Shell;
 import org.apache.hadoop.util.Shell.ExitCodeException;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants.Environment;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -399,10 +400,14 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     // set up the rest of the container
     List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
     containerLaunchContext.setCommands(commands);
-    StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
-    startRequest.setContainerToken(createContainerToken(cId));
-    containerManager.startContainer(startRequest);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          createContainerToken(cId));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
 
     int timeoutSecs = 0;
     while (!processStartFile.exists() && timeoutSecs++ < 20) {
@@ -465,18 +470,20 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     Assert.assertTrue("Process is not alive!",
       DefaultContainerExecutor.containerIsAlive(pid));
 
-    StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
-    stopRequest.setContainerId(cId);
-    containerManager.stopContainer(stopRequest);
+    // Now test the stop functionality.
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cId);
+    StopContainersRequest stopRequest =
+        StopContainersRequest.newInstance(containerIds);
+    containerManager.stopContainers(stopRequest);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
         ContainerState.COMPLETE);
 
-    GetContainerStatusRequest gcsRequest = 
-        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-    gcsRequest.setContainerId(cId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
     ContainerStatus containerStatus = 
-        containerManager.getContainerStatus(gcsRequest).getStatus();
+        containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
     int expectedExitCode = Shell.WINDOWS ? ExitCode.FORCE_KILLED.getExitCode() :
       ExitCode.TERMINATED.getExitCode();
     Assert.assertEquals(expectedExitCode, containerStatus.getExitStatus());
@@ -544,11 +551,15 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     List<String> commands = Arrays.asList(Shell.getRunScriptCommand(scriptFile));
     containerLaunchContext.setCommands(commands);
     Token containerToken = createContainerToken(cId);
-    StartContainerRequest startRequest =
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
-    startRequest.setContainerToken(containerToken);
-    containerManager.startContainer(startRequest);
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          containerToken);
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
 
     int timeoutSecs = 0;
     while (!processStartFile.exists() && timeoutSecs++ < 20) {
@@ -559,19 +570,22 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
         processStartFile.exists());
 
     // Now test the stop functionality.
-    StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
-    stopRequest.setContainerId(cId);
-    containerManager.stopContainer(stopRequest);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cId);
+    StopContainersRequest stopRequest =
+        StopContainersRequest.newInstance(containerIds);
+    containerManager.stopContainers(stopRequest);
 
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
         ContainerState.COMPLETE);
 
     // container stop sends a sigterm followed by a sigkill
-    GetContainerStatusRequest gcsRequest = 
-        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-    gcsRequest.setContainerId(cId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
+    
     ContainerStatus containerStatus = 
-        containerManager.getContainerStatus(gcsRequest).getStatus();
+        containerManager.getContainerStatuses(gcsRequest)
+          .getContainerStatuses().get(0);
     Assert.assertEquals(ExitCode.FORCE_KILLED.getExitCode(),
         containerStatus.getExitStatus());
 

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/MockLocalResourceStatus.java

@@ -18,8 +18,8 @@
 package org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer;
 
 import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.URL;
-import org.apache.hadoop.yarn.server.api.records.SerializedException;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.LocalResourceStatus;
 import org.apache.hadoop.yarn.server.nodemanager.api.protocolrecords.ResourceStatusType;
 

+ 3 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestResourceLocalizationService.java

@@ -80,6 +80,7 @@ import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.hadoop.yarn.api.records.LocalResourceType;
 import org.apache.hadoop.yarn.api.records.LocalResourceVisibility;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.URL;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.event.AsyncDispatcher;
@@ -1013,8 +1014,8 @@ public class TestResourceLocalizationService {
       String localizerId, LocalResourceRequest req) {
     LocalizerStatus status = createLocalizerStatus(localizerId);
     LocalResourceStatus resourceStatus = new LocalResourceStatusPBImpl();
-    resourceStatus.setException(YarnServerBuilderUtils
-        .newSerializedException(new YarnException("test")));
+    resourceStatus.setException(SerializedException
+      .newInstance(new YarnException("test")));
     resourceStatus.setStatus(ResourceStatusType.FETCH_FAILURE);
     resourceStatus.setResource(req);
     status.addResourceStatus(resourceStatus);

+ 12 - 7
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

@@ -63,6 +63,7 @@ import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -809,14 +810,18 @@ public class TestLogAggregationService extends BaseContainerManagerTest {
     commands.add("/bin/bash");
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
-    StartContainerRequest startRequest =
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
-    startRequest.setContainerToken(TestContainerManager.createContainerToken(
-      cId, DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
-      context.getContainerTokenSecretManager()));
-    this.containerManager.startContainer(startRequest);
 
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          TestContainerManager.createContainerToken(
+            cId, DUMMY_RM_IDENTIFIER, context.getNodeId(), user,
+            context.getContainerTokenSecretManager()));
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    this.containerManager.startContainers(allRequests);
+    
     BaseContainerManagerTest.waitForContainerState(this.containerManager,
         cId, ContainerState.COMPLETE);
 

+ 15 - 10
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

@@ -40,8 +40,9 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnsupportedFileSystemException;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -227,9 +228,6 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     commands.add(scriptFile.getAbsolutePath());
     containerLaunchContext.setCommands(commands);
     Resource r = BuilderUtils.newResource(8 * 1024 * 1024, 1);
-    StartContainerRequest startRequest =
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    startRequest.setContainerLaunchContext(containerLaunchContext);
     ContainerTokenIdentifier containerIdentifier =
         new ContainerTokenIdentifier(cId, context.getNodeId().toString(), user,
           r, System.currentTimeMillis() + 120000, 123, DUMMY_RM_IDENTIFIER);
@@ -237,8 +235,14 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
         BuilderUtils.newContainerToken(context.getNodeId(),
           containerManager.getContext().getContainerTokenSecretManager()
             .createPassword(containerIdentifier), containerIdentifier);
-    startRequest.setContainerToken(containerToken);
-    containerManager.startContainer(startRequest);
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(containerLaunchContext,
+          containerToken);
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+    containerManager.startContainers(allRequests);
     
     int timeoutSecs = 0;
     while (!processStartFile.exists() && timeoutSecs++ < 20) {
@@ -260,11 +264,12 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     BaseContainerManagerTest.waitForContainerState(containerManager, cId,
         ContainerState.COMPLETE, 60);
 
-    GetContainerStatusRequest gcsRequest =
-        recordFactory.newRecordInstance(GetContainerStatusRequest.class);
-    gcsRequest.setContainerId(cId);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(cId);
+    GetContainerStatusesRequest gcsRequest =
+        GetContainerStatusesRequest.newInstance(containerIds);
     ContainerStatus containerStatus =
-        containerManager.getContainerStatus(gcsRequest).getStatus();
+        containerManager.getContainerStatuses(gcsRequest).getContainerStatuses().get(0);
     Assert.assertEquals(ExitCode.TERMINATED.getExitCode(),
         containerStatus.getExitStatus());
     String expectedMsgPattern =

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

@@ -22,6 +22,8 @@ import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.nio.ByteBuffer;
 import java.security.PrivilegedAction;
+import java.util.ArrayList;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
@@ -33,11 +35,15 @@ import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.util.StringUtils;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ApplicationSubmissionContext;
 import org.apache.hadoop.yarn.api.records.Container;
@@ -46,8 +52,6 @@ import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.exceptions.YarnException;
-import org.apache.hadoop.yarn.factories.RecordFactory;
-import org.apache.hadoop.yarn.factory.providers.RecordFactoryProvider;
 import org.apache.hadoop.yarn.ipc.YarnRPC;
 import org.apache.hadoop.yarn.security.AMRMTokenIdentifier;
 import org.apache.hadoop.yarn.server.resourcemanager.RMContext;
@@ -68,8 +72,6 @@ public class AMLauncher implements Runnable {
 
   private final RMAppAttempt application;
   private final Configuration conf;
-  private final RecordFactory recordFactory = 
-      RecordFactoryProvider.getRecordFactory(null);
   private final AMLauncherEventType eventType;
   private final RMContext rmContext;
   private final Container masterContainer;
@@ -102,22 +104,42 @@ public class AMLauncher implements Runnable {
         + " for AM " + application.getAppAttemptId());  
     ContainerLaunchContext launchContext =
         createAMContainerLaunchContext(applicationContext, masterContainerID);
-    StartContainerRequest request = 
-        recordFactory.newRecordInstance(StartContainerRequest.class);
-    request.setContainerLaunchContext(launchContext);
-    request.setContainerToken(masterContainer.getContainerToken());
-    containerMgrProxy.startContainer(request);
-    LOG.info("Done launching container " + masterContainer
-        + " for AM " + application.getAppAttemptId());
+
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(launchContext,
+          masterContainer.getContainerToken());
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
+
+    StartContainersResponse response =
+        containerMgrProxy.startContainers(allRequests);
+    if (response.getFailedRequests() != null
+        && response.getFailedRequests().containsKey(masterContainerID)) {
+      Throwable t =
+          response.getFailedRequests().get(masterContainerID).deSerialize();
+      parseAndThrowException(t);
+    } else {
+      LOG.info("Done launching container " + masterContainer + " for AM "
+          + application.getAppAttemptId());
+    }
   }
   
   private void cleanup() throws IOException, YarnException {
     connect();
     ContainerId containerId = masterContainer.getId();
-    StopContainerRequest stopRequest = 
-        recordFactory.newRecordInstance(StopContainerRequest.class);
-    stopRequest.setContainerId(containerId);
-    containerMgrProxy.stopContainer(stopRequest);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
+    StopContainersRequest stopRequest =
+        StopContainersRequest.newInstance(containerIds);
+    StopContainersResponse response =
+        containerMgrProxy.stopContainers(stopRequest);
+    if (response.getFailedRequests() != null
+        && response.getFailedRequests().containsKey(containerId)) {
+      Throwable t = response.getFailedRequests().get(containerId).deSerialize();
+      parseAndThrowException(t);
+    }
   }
 
   // Protected. For tests.
@@ -254,4 +276,15 @@ public class AMLauncher implements Runnable {
       break;
     }
   }
+
+  private void parseAndThrowException(Throwable t) throws YarnException,
+      IOException {
+    if (t instanceof YarnException) {
+      throw (YarnException) t;
+    } else if (t instanceof InvalidToken) {
+      throw (InvalidToken) t;
+    } else {
+      throw (IOException) t;
+    }
+  }
 }

+ 16 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/Application.java

@@ -35,7 +35,8 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.SubmitApplicationRequest;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
@@ -212,9 +213,11 @@ public class Application {
     NodeManager nodeManager = task.getNodeManager();
     ContainerId containerId = task.getContainerId();
     task.stop();
-    StopContainerRequest stopRequest = recordFactory.newRecordInstance(StopContainerRequest.class);
-    stopRequest.setContainerId(containerId);
-    nodeManager.stopContainer(stopRequest);
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
+    StopContainersRequest stopRequest =
+        StopContainersRequest.newInstance(containerIds);
+    nodeManager.stopContainers(stopRequest);
     
     Resources.subtractFrom(used, requestSpec.get(task.getPriority()));
     
@@ -339,10 +342,15 @@ public class Application {
             updateResourceRequests(requests.get(priority), type, task);
 
             // Launch the container
-            StartContainerRequest startRequest = recordFactory.newRecordInstance(StartContainerRequest.class);
-            startRequest.setContainerLaunchContext(createCLC());
-            startRequest.setContainerToken(container.getContainerToken());
-            nodeManager.startContainer(startRequest);
+            StartContainerRequest scRequest =
+                StartContainerRequest.newInstance(createCLC(),
+                  container.getContainerToken());
+            List<StartContainerRequest> list =
+                new ArrayList<StartContainerRequest>();
+            list.add(scRequest);
+            StartContainersRequest allRequests =
+                StartContainersRequest.newInstance(list);
+            nodeManager.startContainers(allRequests);
             break;
           }
         }

+ 113 - 110
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/NodeManager.java

@@ -31,12 +31,13 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -158,61 +159,60 @@ public class NodeManager implements ContainerManagementProtocol {
   }
 
   @Override
-  synchronized public StartContainerResponse startContainer(
-      StartContainerRequest request) 
+  synchronized public StartContainersResponse startContainers(
+      StartContainersRequest requests) 
   throws YarnException {
 
-    Token containerToken = request.getContainerToken();
-    ContainerTokenIdentifier tokenId = null;
+    for (StartContainerRequest request : requests.getStartContainerRequests()) {
+      Token containerToken = request.getContainerToken();
+      ContainerTokenIdentifier tokenId = null;
 
-    try {
-      tokenId = BuilderUtils.newContainerTokenIdentifier(containerToken);
-    } catch (IOException e) {
-      throw RPCUtil.getRemoteException(e);
-    }
+      try {
+        tokenId = BuilderUtils.newContainerTokenIdentifier(containerToken);
+      } catch (IOException e) {
+        throw RPCUtil.getRemoteException(e);
+      }
 
-    ContainerId containerID = tokenId.getContainerID();
-    ApplicationId applicationId =
-        containerID.getApplicationAttemptId().getApplicationId();
+      ContainerId containerID = tokenId.getContainerID();
+      ApplicationId applicationId =
+          containerID.getApplicationAttemptId().getApplicationId();
 
-    List<Container> applicationContainers = containers.get(applicationId);
-    if (applicationContainers == null) {
-      applicationContainers = new ArrayList<Container>();
-      containers.put(applicationId, applicationContainers);
-    }
-    
-    // Sanity check
-    for (Container container : applicationContainers) {
-      if (container.getId().compareTo(containerID)
-          == 0) {
-        throw new IllegalStateException(
-            "Container " + containerID +
-            " already setup on node " + containerManagerAddress);
+      List<Container> applicationContainers = containers.get(applicationId);
+      if (applicationContainers == null) {
+        applicationContainers = new ArrayList<Container>();
+        containers.put(applicationId, applicationContainers);
+      }
+
+      // Sanity check
+      for (Container container : applicationContainers) {
+        if (container.getId().compareTo(containerID) == 0) {
+          throw new IllegalStateException("Container " + containerID
+              + " already setup on node " + containerManagerAddress);
+        }
       }
-    }
 
-    Container container =
-        BuilderUtils.newContainer(containerID,
-            this.nodeId, nodeHttpAddress,
-            tokenId.getResource(),
-            null, null                                 // DKDC - Doesn't matter
+      Container container =
+          BuilderUtils.newContainer(containerID, this.nodeId, nodeHttpAddress,
+            tokenId.getResource(), null, null // DKDC - Doesn't matter
             );
 
-    ContainerStatus containerStatus =
-        BuilderUtils.newContainerStatus(container.getId(), ContainerState.NEW,
-            "", -1000);
-    applicationContainers.add(container);
-    containerStatusMap.put(container, containerStatus);
-    Resources.subtractFrom(available, tokenId.getResource());
-    Resources.addTo(used, tokenId.getResource());
-    
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("startContainer:" + " node=" + containerManagerAddress
-        + " application=" + applicationId + " container=" + container
-        + " available=" + available + " used=" + used);
-    }
+      ContainerStatus containerStatus =
+          BuilderUtils.newContainerStatus(container.getId(),
+            ContainerState.NEW, "", -1000);
+      applicationContainers.add(container);
+      containerStatusMap.put(container, containerStatus);
+      Resources.subtractFrom(available, tokenId.getResource());
+      Resources.addTo(used, tokenId.getResource());
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("startContainer:" + " node=" + containerManagerAddress
+            + " application=" + applicationId + " container=" + container
+            + " available=" + available + " used=" + used);
+      }
 
-    StartContainerResponse response = recordFactory.newRecordInstance(StartContainerResponse.class);
+    }
+    StartContainersResponse response =
+        StartContainersResponse.newInstance(null, null, null);
     return response;
   }
 
@@ -225,76 +225,79 @@ public class NodeManager implements ContainerManagementProtocol {
   }
   
   @Override
-  synchronized public StopContainerResponse stopContainer(StopContainerRequest request) 
+  synchronized public StopContainersResponse stopContainers(StopContainersRequest request) 
   throws YarnException {
-    ContainerId containerID = request.getContainerId();
-    String applicationId = String.valueOf(
-        containerID.getApplicationAttemptId().getApplicationId().getId());
-    
-    // Mark the container as COMPLETE
-    List<Container> applicationContainers = containers.get(applicationId);
-    for (Container c : applicationContainers) {
-      if (c.getId().compareTo(containerID) == 0) {
-        ContainerStatus containerStatus = containerStatusMap.get(c);
-        containerStatus.setState(ContainerState.COMPLETE);
-        containerStatusMap.put(c, containerStatus);
+    for (ContainerId containerID : request.getContainerIds()) {
+      String applicationId =
+          String.valueOf(containerID.getApplicationAttemptId()
+            .getApplicationId().getId());
+
+      // Mark the container as COMPLETE
+      List<Container> applicationContainers = containers.get(applicationId);
+      for (Container c : applicationContainers) {
+        if (c.getId().compareTo(containerID) == 0) {
+          ContainerStatus containerStatus = containerStatusMap.get(c);
+          containerStatus.setState(ContainerState.COMPLETE);
+          containerStatusMap.put(c, containerStatus);
+        }
       }
-    }
-    
-    // Send a heartbeat
-    try {
-      heartbeat();
-    } catch (IOException ioe) {
-      throw RPCUtil.getRemoteException(ioe);
-    }
-    
-    // Remove container and update status
-    int ctr = 0;
-    Container container = null;
-    for (Iterator<Container> i=applicationContainers.iterator(); i.hasNext();) {
-      container = i.next();
-      if (container.getId().compareTo(containerID) == 0) {
-        i.remove();
-        ++ctr;
+
+      // Send a heartbeat
+      try {
+        heartbeat();
+      } catch (IOException ioe) {
+        throw RPCUtil.getRemoteException(ioe);
       }
-    }
-    
-    if (ctr != 1) {
-      throw new IllegalStateException("Container " + containerID + 
-          " stopped " + ctr + " times!");
-    }
-    
-    Resources.addTo(available, container.getResource());
-    Resources.subtractFrom(used, container.getResource());
 
-    if(LOG.isDebugEnabled()) {
-      LOG.debug("stopContainer:" + " node=" + containerManagerAddress
-        + " application=" + applicationId + " container=" + containerID
-        + " available=" + available + " used=" + used);
-    }
+      // Remove container and update status
+      int ctr = 0;
+      Container container = null;
+      for (Iterator<Container> i = applicationContainers.iterator(); i
+        .hasNext();) {
+        container = i.next();
+        if (container.getId().compareTo(containerID) == 0) {
+          i.remove();
+          ++ctr;
+        }
+      }
 
-    StopContainerResponse response = recordFactory.newRecordInstance(StopContainerResponse.class);
-    return response;
+      if (ctr != 1) {
+        throw new IllegalStateException("Container " + containerID
+            + " stopped " + ctr + " times!");
+      }
+
+      Resources.addTo(available, container.getResource());
+      Resources.subtractFrom(used, container.getResource());
+
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("stopContainer:" + " node=" + containerManagerAddress
+            + " application=" + applicationId + " container=" + containerID
+            + " available=" + available + " used=" + used);
+      }
+    }
+    return StopContainersResponse.newInstance(null,null);
   }
 
   @Override
-  synchronized public GetContainerStatusResponse getContainerStatus(GetContainerStatusRequest request) throws YarnException {
-    ContainerId containerId = request.getContainerId();
-    List<Container> appContainers = 
-        containers.get(
-            containerId.getApplicationAttemptId().getApplicationId());
-    Container container = null;
-    for (Container c : appContainers) {
-      if (c.getId().equals(containerId)) {
-        container = c;
+  synchronized public GetContainerStatusesResponse getContainerStatuses(
+      GetContainerStatusesRequest request) throws YarnException {
+    List<ContainerStatus> statuses = new ArrayList<ContainerStatus>();
+    for (ContainerId containerId : request.getContainerIds()) {
+      List<Container> appContainers =
+          containers.get(containerId.getApplicationAttemptId()
+            .getApplicationId());
+      Container container = null;
+      for (Container c : appContainers) {
+        if (c.getId().equals(containerId)) {
+          container = c;
+        }
+      }
+      if (container != null
+          && containerStatusMap.get(container).getState() != null) {
+        statuses.add(containerStatusMap.get(container));
       }
     }
-    GetContainerStatusResponse response = 
-        recordFactory.newRecordInstance(GetContainerStatusResponse.class);
-    if (container != null && containerStatusMap.get(container).getState() != null) {
-      response.setStatus(containerStatusMap.get(container));
-    }
-    return response;
+    return GetContainerStatusesResponse.newInstance(statuses, null);
   }
 
   public static org.apache.hadoop.yarn.server.api.records.NodeStatus 

+ 15 - 15
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestAMAuthorization.java

@@ -36,14 +36,14 @@ import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.ApplicationMasterProtocol;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
@@ -98,23 +98,23 @@ public class TestAMAuthorization {
     }
 
     @Override
-    public StartContainerResponse
-        startContainer(StartContainerRequest request)
+    public StartContainersResponse
+        startContainers(StartContainersRequest request)
             throws YarnException {
-      containerTokens = request.getContainerLaunchContext().getTokens();
-      return null;
+      containerTokens = request.getStartContainerRequests().get(0).getContainerLaunchContext().getTokens();
+      return StartContainersResponse.newInstance(null, null, null);
     }
 
     @Override
-    public StopContainerResponse stopContainer(StopContainerRequest request)
+    public StopContainersResponse stopContainers(StopContainersRequest request)
         throws YarnException {
-      return null;
+      return StopContainersResponse.newInstance(null, null);
     }
 
     @Override
-    public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request) throws YarnException {
-      return null;
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request) throws YarnException {
+      return GetContainerStatusesResponse.newInstance(null, null);
     }
 
     public Credentials getContainerCredentials() throws IOException {

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

@@ -19,7 +19,10 @@
 package org.apache.hadoop.yarn.server.resourcemanager;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.commons.logging.Log;
@@ -27,16 +30,18 @@ import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.AllocateResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StartContainerResponse;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerRequest;
-import org.apache.hadoop.yarn.api.protocolrecords.StopContainerResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StopContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerState;
 import org.apache.hadoop.yarn.api.records.ResourceRequest;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -69,9 +74,10 @@ public class TestApplicationMasterLauncher {
     int maxAppAttempts;
 
     @Override
-    public StartContainerResponse
-        startContainer(StartContainerRequest request)
+    public StartContainersResponse
+        startContainers(StartContainersRequest requests)
             throws YarnException {
+      StartContainerRequest request = requests.getStartContainerRequests().get(0);
       LOG.info("Container started by MyContainerManager: " + request);
       launched = true;
       Map<String, String> env =
@@ -95,11 +101,13 @@ public class TestApplicationMasterLauncher {
           Long.parseLong(env.get(ApplicationConstants.APP_SUBMIT_TIME_ENV));
       maxAppAttempts =
           Integer.parseInt(env.get(ApplicationConstants.MAX_APP_ATTEMPTS_ENV));
-      return null;
+      return StartContainersResponse.newInstance(
+        new HashMap<String, ByteBuffer>(), new ArrayList<ContainerId>(),
+        new HashMap<ContainerId, SerializedException>());
     }
 
     @Override
-    public StopContainerResponse stopContainer(StopContainerRequest request)
+    public StopContainersResponse stopContainers(StopContainersRequest request)
         throws YarnException {
       LOG.info("Container cleaned up by MyContainerManager");
       cleanedup = true;
@@ -107,11 +115,10 @@ public class TestApplicationMasterLauncher {
     }
 
     @Override
-    public GetContainerStatusResponse getContainerStatus(
-        GetContainerStatusRequest request) throws YarnException {
+    public GetContainerStatusesResponse getContainerStatuses(
+        GetContainerStatusesRequest request) throws YarnException {
       return null;
     }
-
   }
 
   @Test

+ 7 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/security/TestClientToAMTokens.java

@@ -19,7 +19,9 @@
 package org.apache.hadoop.yarn.server.resourcemanager.security;
 
 import static org.junit.Assert.fail;
+import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.lang.annotation.Annotation;
@@ -50,6 +52,8 @@ import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportRequest;
 import org.apache.hadoop.yarn.api.protocolrecords.GetApplicationReportResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.RegisterApplicationMasterResponse;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
 import org.apache.hadoop.yarn.event.Dispatcher;
@@ -158,6 +162,9 @@ public class TestClientToAMTokens {
 
     ContainerManagementProtocol containerManager =
         mock(ContainerManagementProtocol.class);
+    StartContainersResponse mockResponse = mock(StartContainersResponse.class);
+    when(containerManager.startContainers((StartContainersRequest) any()))
+      .thenReturn(mockResponse);
     final DrainDispatcher dispatcher = new DrainDispatcher();
 
     MockRM rm = new MockRMWithCustomAMLauncher(conf, containerManager) {

+ 39 - 14
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

@@ -23,6 +23,8 @@ import static org.junit.Assert.fail;
 import java.io.IOException;
 import java.net.InetSocketAddress;
 import java.security.PrivilegedAction;
+import java.util.ArrayList;
+import java.util.List;
 
 import junit.framework.Assert;
 
@@ -32,15 +34,20 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.SecretManager.InvalidToken;
 import org.apache.hadoop.yarn.api.ContainerManagementProtocol;
-import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.GetContainerStatusesResponse;
 import org.apache.hadoop.yarn.api.protocolrecords.StartContainerRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersRequest;
+import org.apache.hadoop.yarn.api.protocolrecords.StartContainersResponse;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.SerializedException;
 import org.apache.hadoop.yarn.api.records.Token;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
@@ -328,18 +335,21 @@ public class TestContainerManagerSecurity {
           ContainerId containerId,
           ApplicationAttemptId appAttemptId, NodeId nodeId,
           boolean isExceptionExpected) throws Exception {
-    GetContainerStatusRequest request =
-        Records.newRecord(GetContainerStatusRequest.class);
-    request.setContainerId(containerId);
-    
+    List<ContainerId> containerIds = new ArrayList<ContainerId>();
+    containerIds.add(containerId);
+    GetContainerStatusesRequest request =
+        GetContainerStatusesRequest.newInstance(containerIds);
     ContainerManagementProtocol proxy = null;
-    
     try {
       proxy =
           getContainerManagementProtocolProxy(rpc, nmToken, nodeId,
               appAttemptId.toString());
-      proxy.getContainerStatus(request);
-      
+      GetContainerStatusesResponse statuses = proxy.getContainerStatuses(request);
+      if (statuses.getFailedRequests() != null
+          && statuses.getFailedRequests().containsKey(containerId)) {
+        parseAndThrowException(statuses.getFailedRequests().get(containerId)
+          .deSerialize());
+      }
     } finally {
       if (proxy != null) {
         rpc.stopProxy(proxy, conf);
@@ -352,17 +362,21 @@ public class TestContainerManagerSecurity {
       org.apache.hadoop.yarn.api.records.Token containerToken,
       NodeId nodeId, String user) throws Exception {
 
-    StartContainerRequest request =
-        Records.newRecord(StartContainerRequest.class);
-    request.setContainerToken(containerToken);
     ContainerLaunchContext context =
         Records.newRecord(ContainerLaunchContext.class);
-    request.setContainerLaunchContext(context);
-
+    StartContainerRequest scRequest =
+        StartContainerRequest.newInstance(context,containerToken);
+    List<StartContainerRequest> list = new ArrayList<StartContainerRequest>();
+    list.add(scRequest);
+    StartContainersRequest allRequests =
+        StartContainersRequest.newInstance(list);
     ContainerManagementProtocol proxy = null;
     try {
       proxy = getContainerManagementProtocolProxy(rpc, nmToken, nodeId, user);
-      proxy.startContainer(request);
+      StartContainersResponse response = proxy.startContainers(allRequests);
+      for(SerializedException ex : response.getFailedRequests().values()){
+        parseAndThrowException(ex.deSerialize());
+      }
     } finally {
       if (proxy != null) {
         rpc.stopProxy(proxy, conf);
@@ -370,6 +384,17 @@ public class TestContainerManagerSecurity {
     }
   }
 
+  private void parseAndThrowException(Throwable t) throws YarnException,
+      IOException {
+    if (t instanceof YarnException) {
+      throw (YarnException) t;
+    } else if (t instanceof InvalidToken) {
+      throw (InvalidToken) t;
+    } else {
+      throw (IOException) t;
+    }
+  }
+
   protected ContainerManagementProtocol getContainerManagementProtocolProxy(
       final YarnRPC rpc, org.apache.hadoop.yarn.api.records.Token nmToken,
       NodeId nodeId, String user) {