Browse Source

YARN-2830. Add backwords compatible ContainerId.newInstance constructor. Contributed by Jonathan Eagles.

(cherry picked from commit 43cd07b408c6613d2c9aa89203cfa3110d830538)
Arun C. Murthy 10 years ago
parent
commit
175d222bfc
75 changed files with 204 additions and 189 deletions
  1. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
  2. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
  3. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
  4. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  5. 3 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MockJobs.java
  6. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestMRAppMaster.java
  7. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
  8. 11 11
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskAttempt.java
  9. 4 4
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncher.java
  10. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/launcher/TestContainerLauncherImpl.java
  11. 5 5
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/rm/TestRMContainerAllocator.java
  12. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java
  13. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java
  14. 3 0
      hadoop-yarn-project/CHANGES.txt
  15. 14 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
  16. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-applications/hadoop-yarn-applications-unmanaged-am-launcher/src/main/java/org/apache/hadoop/yarn/applications/unmanagedamlauncher/UnmanagedAMLauncher.java
  17. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java
  18. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java
  19. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
  20. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java
  21. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java
  22. 8 8
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java
  23. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java
  24. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java
  25. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestContainerLaunchRPC.java
  26. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/TestRPC.java
  27. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java
  28. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java
  29. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java
  30. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java
  31. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogFormat.java
  32. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogsBlock.java
  33. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java
  34. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStoreTestUtils.java
  35. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java
  36. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java
  37. 5 5
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
  38. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestMemoryApplicationHistoryStore.java
  39. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java
  40. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
  41. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/main/java/org/apache/hadoop/yarn/server/utils/BuilderUtils.java
  42. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
  43. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java
  44. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestRegisterNodeManagerRequest.java
  45. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestEventFlow.java
  46. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestLinuxContainerExecutor.java
  47. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerReboot.java
  48. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerResync.java
  49. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeManagerShutdown.java
  50. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/TestNodeStatusUpdater.java
  51. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestAuxServices.java
  52. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  53. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManagerRecovery.java
  54. 4 4
      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
  55. 1 1
      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
  56. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java
  57. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestApplicationCleanup.java
  58. 3 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestClientRMService.java
  59. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
  60. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRMRestart.java
  61. 4 4
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestResourceTrackerService.java
  62. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestWorkPreservingRMRestart.java
  63. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java
  64. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/MockAsm.java
  65. 11 11
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java
  66. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java
  67. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java
  68. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmapp/attempt/TestRMAppAttemptTransitions.java
  69. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/rmcontainer/TestRMContainerImpl.java
  70. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerApplicationAttempt.java
  71. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java
  72. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java
  73. 16 16
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java
  74. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/fair/TestFairScheduler.java
  75. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-tests/src/test/java/org/apache/hadoop/yarn/server/TestContainerManagerSecurity.java

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

@@ -140,7 +140,7 @@ public class LocalContainerAllocator extends RMCommunicator
       LOG.info("Processing the event " + event.toString());
       // Assign the same container ID as the AM
       ContainerId cID =
-          ContainerId.newInstance(getContext().getApplicationAttemptId(),
+          ContainerId.newContainerId(getContext().getApplicationAttemptId(),
             this.containerId.getContainerId());
       Container container = recordFactory.newRecordInstance(Container.class);
       container.setId(cID);

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java

@@ -716,7 +716,7 @@ public class TestJobHistoryEventHandler {
     ApplicationId appId = ApplicationId.newInstance(200, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     TaskID taskID = TaskID.forName("task_200707121733_0003_m_000005");
     TaskAttemptID taskAttemptID = new TaskAttemptID(taskID, 0);
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);

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

@@ -179,7 +179,7 @@ public class MRApp extends MRAppMaster {
     ApplicationAttemptId appAttemptId =
         getApplicationAttemptId(applicationId, startCount);
     ContainerId containerId =
-        ContainerId.newInstance(appAttemptId, startCount);
+        ContainerId.newContainerId(appAttemptId, startCount);
     return containerId;
   }
 
@@ -565,7 +565,7 @@ public class MRApp extends MRAppMaster {
      @Override
       public void handle(ContainerAllocatorEvent event) {
         ContainerId cId =
-            ContainerId.newInstance(getContext().getApplicationAttemptId(),
+            ContainerId.newContainerId(getContext().getApplicationAttemptId(),
               containerCount++);
         NodeId nodeId = NodeId.newInstance(NM_HOST, NM_PORT);
         Resource resource = Resource.newInstance(1234, 2);
@@ -773,7 +773,7 @@ public class MRApp extends MRAppMaster {
     ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, appAttemptId);
-    return ContainerId.newInstance(applicationAttemptId, containerId);
+    return ContainerId.newContainerId(applicationAttemptId, containerId);
   }
 
   public static ContainerTokenIdentifier newContainerTokenIdentifier(

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

@@ -140,7 +140,7 @@ public class MRAppBenchmark {
                 if (concurrentRunningTasks < maxConcurrentRunningTasks) {
                   event = eventQueue.take();
                   ContainerId cId =
-                      ContainerId.newInstance(getContext()
+                      ContainerId.newContainerId(getContext()
                         .getApplicationAttemptId(), containerCount++);
 
                   //System.out.println("Allocating " + containerCount);
@@ -233,7 +233,7 @@ public class MRAppBenchmark {
                   int numContainers = req.getNumContainers();
                   for (int i = 0; i < numContainers; i++) {
                     ContainerId containerId =
-                        ContainerId.newInstance(
+                        ContainerId.newContainerId(
                           getContext().getApplicationAttemptId(),
                           request.getResponseId() + i);
                     containers.add(Container.newInstance(containerId,

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

@@ -183,7 +183,7 @@ public class MockJobs extends MockApps {
   public static TaskAttemptReport newTaskAttemptReport(TaskAttemptId id) {
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
         id.getTaskId().getJobId().getAppId(), 0);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 0);
     TaskAttemptReport report = Records.newRecord(TaskAttemptReport.class);
     report.setTaskAttemptId(id);
     report
@@ -315,7 +315,7 @@ public class MockJobs extends MockApps {
         ApplicationAttemptId appAttemptId =
             ApplicationAttemptId.newInstance(taid.getTaskId().getJobId()
               .getAppId(), 0);
-        ContainerId id = ContainerId.newInstance(appAttemptId, 0);
+        ContainerId id = ContainerId.newContainerId(appAttemptId, 0);
         return id;
       }
 
@@ -640,7 +640,7 @@ public class MockJobs extends MockApps {
   private static AMInfo createAMInfo(int attempt) {
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
         ApplicationId.newInstance(100, 1), attempt);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     return MRBuilderUtils.newAMInfo(appAttemptId, System.currentTimeMillis(),
         containerId, NM_HOST, NM_PORT, NM_HTTP_PORT);
   }

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

@@ -382,7 +382,7 @@ public class TestMRAppMaster {
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
     ContainerId containerId =
-        ContainerId.newInstance(applicationAttemptId, 546);
+        ContainerId.newContainerId(applicationAttemptId, 546);
     String userName = UserGroupInformation.getCurrentUser().getShortUserName();
 
     // Create staging dir, so MRAppMaster doesn't barf.

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

@@ -253,7 +253,7 @@ import org.junit.Test;
 
      public TestMRApp(ApplicationAttemptId applicationAttemptId, 
          ContainerAllocator allocator) {
-       super(applicationAttemptId, ContainerId.newInstance(
+       super(applicationAttemptId, ContainerId.newContainerId(
            applicationAttemptId, 1), "testhost", 2222, 3333,
            System.currentTimeMillis());
        this.allocator = allocator;

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

@@ -359,7 +359,7 @@ public class TestTaskAttempt{
           new SystemClock(), null);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -415,7 +415,7 @@ public class TestTaskAttempt{
           new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.2", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -472,7 +472,7 @@ public class TestTaskAttempt{
           new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -532,7 +532,7 @@ public class TestTaskAttempt{
           new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -599,7 +599,7 @@ public class TestTaskAttempt{
         new Token(), new Credentials(), new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -649,7 +649,7 @@ public class TestTaskAttempt{
         new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -714,7 +714,7 @@ public class TestTaskAttempt{
         new Token(), new Credentials(), new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -760,7 +760,7 @@ public class TestTaskAttempt{
 	  new SystemClock(), appCtx);
 
 	NodeId nid = NodeId.newInstance("127.0.0.1", 0);
-	ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+	ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
 	Container container = mock(Container.class);
 	when(container.getId()).thenReturn(contId);
 	when(container.getNodeId()).thenReturn(nid);
@@ -830,7 +830,7 @@ public class TestTaskAttempt{
         new Credentials(), new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.2", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -884,7 +884,7 @@ public class TestTaskAttempt{
         new Credentials(), new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.2", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
@@ -941,7 +941,7 @@ public class TestTaskAttempt{
         new Credentials(), new SystemClock(), appCtx);
 
     NodeId nid = NodeId.newInstance("127.0.0.2", 0);
-    ContainerId contId = ContainerId.newInstance(appAttemptId, 3);
+    ContainerId contId = ContainerId.newContainerId(appAttemptId, 3);
     Container container = mock(Container.class);
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);

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

@@ -115,7 +115,7 @@ public class TestContainerLauncher {
 
     containerLauncher.expectedCorePoolSize = ContainerLauncherImpl.INITIAL_POOL_SIZE;
     for (int i = 0; i < 10; i++) {
-      ContainerId containerId = ContainerId.newInstance(appAttemptId, i);
+      ContainerId containerId = ContainerId.newContainerId(appAttemptId, i);
       TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, i);
       containerLauncher.handle(new ContainerLauncherEvent(taskAttemptId,
         containerId, "host" + i + ":1234", null,
@@ -137,7 +137,7 @@ public class TestContainerLauncher {
     Assert.assertEquals(10, containerLauncher.numEventsProcessed.get());
     containerLauncher.finishEventHandling = false;
     for (int i = 0; i < 10; i++) {
-      ContainerId containerId = ContainerId.newInstance(appAttemptId,
+      ContainerId containerId = ContainerId.newContainerId(appAttemptId,
           i + 10);
       TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId,
           i + 10);
@@ -154,7 +154,7 @@ public class TestContainerLauncher {
     // Core pool size should be 21 but the live pool size should be only 11.
     containerLauncher.expectedCorePoolSize = 11 + ContainerLauncherImpl.INITIAL_POOL_SIZE;
     containerLauncher.finishEventHandling = false;
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 21);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 21);
     TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, 21);
     containerLauncher.handle(new ContainerLauncherEvent(taskAttemptId,
       containerId, "host11:1234", null,
@@ -174,7 +174,7 @@ public class TestContainerLauncher {
     JobId jobId = MRBuilderUtils.newJobId(appId, 8);
     TaskId taskId = MRBuilderUtils.newTaskId(jobId, 9, TaskType.MAP);
     TaskAttemptId taskAttemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 10);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 10);
 
     AppContext context = mock(AppContext.class);
     CustomContainerLauncher containerLauncher = new CustomContainerLauncher(

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

@@ -139,7 +139,7 @@ public class TestContainerLauncherImpl {
   
   public static ContainerId makeContainerId(long ts, int appId, int attemptId,
       int id) {
-    return ContainerId.newInstance(
+    return ContainerId.newContainerId(
       ApplicationAttemptId.newInstance(
         ApplicationId.newInstance(ts, appId), attemptId), id);
   }

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

@@ -688,7 +688,7 @@ public class TestRMContainerAllocator {
     rm.sendAMLaunched(appAttemptId);
     rmDispatcher.await();
 
-    MRApp mrApp = new MRApp(appAttemptId, ContainerId.newInstance(
+    MRApp mrApp = new MRApp(appAttemptId, ContainerId.newContainerId(
       appAttemptId, 0), 10, 10, false, this.getClass().getName(), true, 1) {
       @Override
       protected Dispatcher createDispatcher() {
@@ -840,7 +840,7 @@ public class TestRMContainerAllocator {
     rm.sendAMLaunched(appAttemptId);
     rmDispatcher.await();
 
-    MRApp mrApp = new MRApp(appAttemptId, ContainerId.newInstance(
+    MRApp mrApp = new MRApp(appAttemptId, ContainerId.newContainerId(
       appAttemptId, 0), 10, 0, false, this.getClass().getName(), true, 1) {
       @Override
         protected Dispatcher createDispatcher() {
@@ -2021,7 +2021,7 @@ public class TestRMContainerAllocator {
     ApplicationId applicationId = ApplicationId.newInstance(1, 1);
     ApplicationAttemptId applicationAttemptId = ApplicationAttemptId.newInstance(
         applicationId, 1);
-    ContainerId containerId = ContainerId.newInstance(applicationAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(applicationAttemptId, 1);
     ContainerStatus status = ContainerStatus.newInstance(
         containerId, ContainerState.RUNNING, "", 0);
 
@@ -2038,7 +2038,7 @@ public class TestRMContainerAllocator {
         abortedStatus, attemptId);
     Assert.assertEquals(TaskAttemptEventType.TA_KILL, abortedEvent.getType());
     
-    ContainerId containerId2 = ContainerId.newInstance(applicationAttemptId, 2);
+    ContainerId containerId2 = ContainerId.newContainerId(applicationAttemptId, 2);
     ContainerStatus status2 = ContainerStatus.newInstance(containerId2,
         ContainerState.RUNNING, "", 0);
 
@@ -2077,7 +2077,7 @@ public class TestRMContainerAllocator {
     rmDispatcher.await();
 
     MRApp mrApp =
-        new MRApp(appAttemptId, ContainerId.newInstance(appAttemptId, 0), 10,
+        new MRApp(appAttemptId, ContainerId.newContainerId(appAttemptId, 0), 10,
             0, false, this.getClass().getName(), true, 1) {
           @Override
           protected Dispatcher createDispatcher() {

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/test/java/org/apache/hadoop/mapreduce/v2/hs/webapp/TestBlocks.java

@@ -133,7 +133,7 @@ public class TestBlocks {
     ApplicationId appId = ApplicationIdPBImpl.newInstance(0, 5);
     ApplicationAttemptId appAttemptId = ApplicationAttemptIdPBImpl.newInstance(appId, 1);
 
-    ContainerId containerId = ContainerIdPBImpl.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerIdPBImpl.newContainerId(appAttemptId, 1);
     when(attempt.getAssignedContainerID()).thenReturn(containerId);
 
     when(attempt.getAssignedContainerMgrAddress()).thenReturn(

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobsWithHistoryService.java

@@ -169,7 +169,7 @@ public class TestMRJobsWithHistoryService {
     Assert.assertEquals(1, amInfos.size());
     AMInfo amInfo = amInfos.get(0);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(jobId.getAppId(), 1);
-    ContainerId amContainerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId amContainerId = ContainerId.newContainerId(appAttemptId, 1);
     Assert.assertEquals(appAttemptId, amInfo.getAppAttemptId());
     Assert.assertEquals(amContainerId, amInfo.getContainerId());
     Assert.assertTrue(jobReport.getSubmitTime() > 0);

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

@@ -895,6 +895,9 @@ Release 2.6.0 - UNRELEASED
 
     YARN-2607. Fixed issues in TestDistributedShell. (Wangda Tan via vinodkv)
 
+    YARN-2830. Add backwords compatible ContainerId.newInstance constructor.
+    (jeagles via acmurthy) 
+
 Release 2.5.2 - UNRELEASED
 
   INCOMPATIBLE CHANGES

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

@@ -42,7 +42,7 @@ public abstract class ContainerId implements Comparable<ContainerId>{
 
   @Private
   @Unstable
-  public static ContainerId newInstance(ApplicationAttemptId appAttemptId,
+  public static ContainerId newContainerId(ApplicationAttemptId appAttemptId,
       long containerId) {
     ContainerId id = Records.newRecord(ContainerId.class);
     id.setContainerId(containerId);
@@ -51,6 +51,18 @@ public abstract class ContainerId implements Comparable<ContainerId>{
     return id;
   }
 
+  @Private
+  @Deprecated
+  @Unstable
+  public static ContainerId newInstance(ApplicationAttemptId appAttemptId,
+      int containerId) {
+    ContainerId id = Records.newRecord(ContainerId.class);
+    id.setContainerId(containerId);
+    id.setApplicationAttemptId(appAttemptId);
+    id.build();
+    return id;
+  }
+
   /**
    * Get the <code>ApplicationAttemptId</code> of the application to which the
    * <code>Container</code> was assigned.
@@ -214,7 +226,7 @@ public abstract class ContainerId implements Comparable<ContainerId>{
       }
       long id = Long.parseLong(it.next());
       long cid = (epoch << 40) | id;
-      ContainerId containerId = ContainerId.newInstance(appAttemptID, cid);
+      ContainerId containerId = ContainerId.newContainerId(appAttemptID, cid);
       return containerId;
     } catch (NumberFormatException n) {
       throw new IllegalArgumentException("Invalid ContainerId: "

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

@@ -214,7 +214,7 @@ public class UnmanagedAMLauncher {
     if(!setClasspath && classpath!=null) {
       envAMList.add("CLASSPATH="+classpath);
     }
-    ContainerId containerId = ContainerId.newInstance(attemptId, 0);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, 0);
 
     String hostname = InetAddress.getLocalHost().getHostName();
     envAMList.add(Environment.CONTAINER_ID.name() + "=" + containerId);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/ProtocolHATestBase.java

@@ -667,7 +667,7 @@ public abstract class ProtocolHATestBase extends ClientBaseWithFixes {
     }
 
     public ContainerId createFakeContainerId() {
-      return ContainerId.newInstance(createFakeApplicationAttemptId(), 0);
+      return ContainerId.newContainerId(createFakeApplicationAttemptId(), 0);
     }
 
     public YarnClusterMetrics createFakeYarnClusterMetrics() {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestAMRMClientAsync.java

@@ -402,7 +402,7 @@ public class TestAMRMClientAsync {
     ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, appAttemptId);
-    return ContainerId.newInstance(applicationAttemptId, containerId);
+    return ContainerId.newContainerId(applicationAttemptId, containerId);
   }
 
   private class TestCallbackHandler implements AMRMClientAsync.CallbackHandler {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java

@@ -547,7 +547,7 @@ public class TestNMClientAsync {
         ApplicationId.newInstance(System.currentTimeMillis(), 1);
     ApplicationAttemptId attemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(attemptId, i);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, i);
     nodeId = NodeId.newInstance("localhost", 0);
     // Create an empty record
     containerToken = recordFactory.newRecordInstance(Token.class);

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAHSClient.java

@@ -157,9 +157,9 @@ public class TestAHSClient {
     List<ContainerReport> reports = client.getContainers(appAttemptId);
     Assert.assertNotNull(reports);
     Assert.assertEquals(reports.get(0).getContainerId(),
-      (ContainerId.newInstance(appAttemptId, 1)));
+      (ContainerId.newContainerId(appAttemptId, 1)));
     Assert.assertEquals(reports.get(1).getContainerId(),
-      (ContainerId.newInstance(appAttemptId, 2)));
+      (ContainerId.newContainerId(appAttemptId, 2)));
     client.stop();
   }
 
@@ -176,11 +176,11 @@ public class TestAHSClient {
     ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     ContainerReport report = client.getContainerReport(containerId);
     Assert.assertNotNull(report);
     Assert.assertEquals(report.getContainerId().toString(), (ContainerId
-      .newInstance(expectedReports.get(0).getCurrentApplicationAttemptId(), 1))
+      .newContainerId(expectedReports.get(0).getCurrentApplicationAttemptId(), 1))
       .toString());
     client.stop();
   }
@@ -349,7 +349,7 @@ public class TestAHSClient {
             "oUrl",
             "diagnostics",
             YarnApplicationAttemptState.FINISHED,
-            ContainerId.newInstance(
+            ContainerId.newContainerId(
               newApplicationReport.getCurrentApplicationAttemptId(), 1));
       appAttempts.add(attempt);
       ApplicationAttemptReport attempt1 =
@@ -361,7 +361,7 @@ public class TestAHSClient {
             "oUrl",
             "diagnostics",
             YarnApplicationAttemptState.FINISHED,
-            ContainerId.newInstance(
+            ContainerId.newContainerId(
               newApplicationReport.getCurrentApplicationAttemptId(), 2));
       appAttempts.add(attempt1);
       attempts.put(applicationId, appAttempts);
@@ -369,14 +369,14 @@ public class TestAHSClient {
       List<ContainerReport> containerReports = new ArrayList<ContainerReport>();
       ContainerReport container =
           ContainerReport.newInstance(
-            ContainerId.newInstance(attempt.getApplicationAttemptId(), 1),
+            ContainerId.newContainerId(attempt.getApplicationAttemptId(), 1),
             null, NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234,
             5678, "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
       containerReports.add(container);
 
       ContainerReport container1 =
           ContainerReport.newInstance(
-            ContainerId.newInstance(attempt.getApplicationAttemptId(), 2),
+            ContainerId.newContainerId(attempt.getApplicationAttemptId(), 2),
             null, NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234,
             5678, "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
       containerReports.add(container1);

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestAMRMClientOnRMRestart.java

@@ -352,7 +352,7 @@ public class TestAMRMClientOnRMRestart {
     // new NM to represent NM re-register
     nm1 = new MockNM("h1:1234", 10240, rm2.getResourceTrackerService());
 
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     NMContainerStatus containerReport =
         NMContainerStatus.newInstance(containerId, ContainerState.RUNNING,
             Resource.newInstance(1024, 1), "recover container", 0,

+ 8 - 8
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/impl/TestYarnClient.java

@@ -348,9 +348,9 @@ public class TestYarnClient {
     List<ContainerReport> reports = client.getContainers(appAttemptId);
     Assert.assertNotNull(reports);
     Assert.assertEquals(reports.get(0).getContainerId(),
-        (ContainerId.newInstance(appAttemptId, 1)));
+        (ContainerId.newContainerId(appAttemptId, 1)));
     Assert.assertEquals(reports.get(1).getContainerId(),
-        (ContainerId.newInstance(appAttemptId, 2)));
+        (ContainerId.newContainerId(appAttemptId, 2)));
     client.stop();
   }
 
@@ -367,11 +367,11 @@ public class TestYarnClient {
     ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(
         applicationId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     ContainerReport report = client.getContainerReport(containerId);
     Assert.assertNotNull(report);
     Assert.assertEquals(report.getContainerId().toString(),
-        (ContainerId.newInstance(expectedReports.get(0)
+        (ContainerId.newContainerId(expectedReports.get(0)
             .getCurrentApplicationAttemptId(), 1)).toString());
     client.stop();
   }
@@ -481,7 +481,7 @@ public class TestYarnClient {
           "oUrl",
           "diagnostics",
           YarnApplicationAttemptState.FINISHED,
-          ContainerId.newInstance(
+          ContainerId.newContainerId(
               newApplicationReport.getCurrentApplicationAttemptId(), 1));
       appAttempts.add(attempt);
       ApplicationAttemptReport attempt1 = ApplicationAttemptReport.newInstance(
@@ -492,20 +492,20 @@ public class TestYarnClient {
           "oUrl",
           "diagnostics",
           YarnApplicationAttemptState.FINISHED,
-          ContainerId.newInstance(
+          ContainerId.newContainerId(
               newApplicationReport.getCurrentApplicationAttemptId(), 2));
       appAttempts.add(attempt1);
       attempts.put(applicationId, appAttempts);
 
       List<ContainerReport> containerReports = new ArrayList<ContainerReport>();
       ContainerReport container = ContainerReport.newInstance(
-          ContainerId.newInstance(attempt.getApplicationAttemptId(), 1), null,
+          ContainerId.newContainerId(attempt.getApplicationAttemptId(), 1), null,
           NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
           "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
       containerReports.add(container);
 
       ContainerReport container1 = ContainerReport.newInstance(
-          ContainerId.newInstance(attempt.getApplicationAttemptId(), 2), null,
+          ContainerId.newContainerId(attempt.getApplicationAttemptId(), 2), null,
           NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
           "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
       containerReports.add(container1);

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestLogsCLI.java

@@ -172,9 +172,9 @@ public class TestLogsCLI {
     ApplicationId appId = ApplicationIdPBImpl.newInstance(0, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptIdPBImpl.newInstance(appId, 1);
-    ContainerId containerId0 = ContainerIdPBImpl.newInstance(appAttemptId, 0);
-    ContainerId containerId1 = ContainerIdPBImpl.newInstance(appAttemptId, 1);
-    ContainerId containerId2 = ContainerIdPBImpl.newInstance(appAttemptId, 2);
+    ContainerId containerId0 = ContainerIdPBImpl.newContainerId(appAttemptId, 0);
+    ContainerId containerId1 = ContainerIdPBImpl.newContainerId(appAttemptId, 1);
+    ContainerId containerId2 = ContainerIdPBImpl.newContainerId(appAttemptId, 2);
     NodeId nodeId = NodeId.newInstance("localhost", 1234);
 
     // create local logs

+ 7 - 7
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/cli/TestYarnCLI.java

@@ -146,7 +146,7 @@ public class TestYarnCLI {
         applicationId, 1);
     ApplicationAttemptReport attemptReport = ApplicationAttemptReport
         .newInstance(attemptId, "host", 124, "url", "oUrl", "diagnostics",
-            YarnApplicationAttemptState.FINISHED, ContainerId.newInstance(
+            YarnApplicationAttemptState.FINISHED, ContainerId.newContainerId(
                 attemptId, 1));
     when(
         client
@@ -182,11 +182,11 @@ public class TestYarnCLI {
         applicationId, 2);
     ApplicationAttemptReport attemptReport = ApplicationAttemptReport
         .newInstance(attemptId, "host", 124, "url", "oUrl", "diagnostics",
-            YarnApplicationAttemptState.FINISHED, ContainerId.newInstance(
+            YarnApplicationAttemptState.FINISHED, ContainerId.newContainerId(
                 attemptId, 1));
     ApplicationAttemptReport attemptReport1 = ApplicationAttemptReport
         .newInstance(attemptId1, "host", 124, "url", "oUrl", "diagnostics",
-            YarnApplicationAttemptState.FINISHED, ContainerId.newInstance(
+            YarnApplicationAttemptState.FINISHED, ContainerId.newContainerId(
                 attemptId1, 1));
     List<ApplicationAttemptReport> reports = new ArrayList<ApplicationAttemptReport>();
     reports.add(attemptReport);
@@ -223,7 +223,7 @@ public class TestYarnCLI {
     ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
         applicationId, 1);
-    ContainerId containerId = ContainerId.newInstance(attemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
     ContainerReport container = ContainerReport.newInstance(containerId, null,
         NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
         "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
@@ -255,8 +255,8 @@ public class TestYarnCLI {
     ApplicationId applicationId = ApplicationId.newInstance(1234, 5);
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
         applicationId, 1);
-    ContainerId containerId = ContainerId.newInstance(attemptId, 1);
-    ContainerId containerId1 = ContainerId.newInstance(attemptId, 2);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
+    ContainerId containerId1 = ContainerId.newContainerId(attemptId, 2);
     ContainerReport container = ContainerReport.newInstance(containerId, null,
         NodeId.newInstance("host", 1234), Priority.UNDEFINED, 1234, 5678,
         "diagnosticInfo", "logURL", 0, ContainerState.COMPLETE);
@@ -766,7 +766,7 @@ public class TestYarnCLI {
         sysOutStream.toString());
 
     sysOutStream.reset();
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 7);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 7);
     result = cli.run(
         new String[] { "container", "-status", containerId.toString(), "args" });
     verify(spyCli).printUsage(any(String.class), any(Options.class));

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

@@ -97,7 +97,7 @@ public class TestContainerLaunchRPC {
       ApplicationAttemptId applicationAttemptId =
           ApplicationAttemptId.newInstance(applicationId, 0);
       ContainerId containerId =
-          ContainerId.newInstance(applicationAttemptId, 100);
+          ContainerId.newContainerId(applicationAttemptId, 100);
       NodeId nodeId = NodeId.newInstance("localhost", 1234);
       Resource resource = Resource.newInstance(1234, 2);
       ContainerTokenIdentifier containerTokenIdentifier =

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

@@ -124,7 +124,7 @@ public class TestRPC {
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 0);
     ContainerId containerId =
-        ContainerId.newInstance(applicationAttemptId, 100);
+        ContainerId.newContainerId(applicationAttemptId, 100);
     NodeId nodeId = NodeId.newInstance("localhost", 1234);
     Resource resource = Resource.newInstance(1234, 2);
     ContainerTokenIdentifier containerTokenIdentifier =

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerId.java

@@ -79,6 +79,6 @@ public class TestContainerId {
     ApplicationId applicationId = ApplicationId.newInstance(timestamp, appId);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, appAttemptId);
-    return ContainerId.newInstance(applicationAttemptId, containerId);
+    return ContainerId.newContainerId(applicationAttemptId, containerId);
   }
 }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceDecrease.java

@@ -33,7 +33,7 @@ public class TestContainerResourceDecrease {
   @Test
   public void testResourceDecreaseContext() {
     ContainerId containerId = ContainerId
-        .newInstance(ApplicationAttemptId.newInstance(
+        .newContainerId(ApplicationAttemptId.newInstance(
             ApplicationId.newInstance(1234, 3), 3), 7);
     Resource resource = Resource.newInstance(1023, 3);
     ContainerResourceDecrease ctx = ContainerResourceDecrease.newInstance(

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncrease.java

@@ -38,7 +38,7 @@ public class TestContainerResourceIncrease {
     byte[] identifier = new byte[] { 1, 2, 3, 4 };
     Token token = Token.newInstance(identifier, "", "".getBytes(), "");
     ContainerId containerId = ContainerId
-        .newInstance(ApplicationAttemptId.newInstance(
+        .newContainerId(ApplicationAttemptId.newInstance(
             ApplicationId.newInstance(1234, 3), 3), 7);
     Resource resource = Resource.newInstance(1023, 3);
     ContainerResourceIncrease ctx = ContainerResourceIncrease.newInstance(

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/api/TestContainerResourceIncreaseRequest.java

@@ -33,7 +33,7 @@ public class TestContainerResourceIncreaseRequest {
   @Test
   public void ContainerResourceIncreaseRequest() {
     ContainerId containerId = ContainerId
-        .newInstance(ApplicationAttemptId.newInstance(
+        .newContainerId(ApplicationAttemptId.newInstance(
             ApplicationId.newInstance(1234, 3), 3), 7);
     Resource resource = Resource.newInstance(1023, 3);
     ContainerResourceIncreaseRequest context = ContainerResourceIncreaseRequest

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogFormat.java

@@ -295,7 +295,7 @@ public class TestAggregatedLogFormat {
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 1);
     ContainerId testContainerId1 =
-        ContainerId.newInstance(applicationAttemptId, 1);
+        ContainerId.newContainerId(applicationAttemptId, 1);
     Path appDir =
         new Path(srcFileRoot, testContainerId1.getApplicationAttemptId()
             .getApplicationId().toString());

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/logaggregation/TestAggregatedLogsBlock.java

@@ -207,7 +207,7 @@ public class TestAggregatedLogsBlock {
       throws Exception {
     ApplicationId appId =  ApplicationIdPBImpl.newInstance(0, 1);
     ApplicationAttemptId appAttemptId =  ApplicationAttemptIdPBImpl.newInstance(appId, 1);
-    ContainerId containerId = ContainerIdPBImpl.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerIdPBImpl.newContainerId(appAttemptId, 1);
 
     String path = "target/logs/" + user
         + "/logs/application_0_0001/localhost_1234";

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/test/java/org/apache/hadoop/yarn/security/TestYARNTokenIdentifier.java

@@ -134,7 +134,7 @@ public class TestYARNTokenIdentifier {
   
   @Test
   public void testContainerTokenIdentifier() throws IOException {
-    ContainerId containerID = ContainerId.newInstance(
+    ContainerId containerID = ContainerId.newContainerId(
         ApplicationAttemptId.newInstance(ApplicationId.newInstance(
             1, 1), 1), 1);
     String hostName = "host0";

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStoreTestUtils.java

@@ -58,7 +58,7 @@ public class ApplicationHistoryStoreTestUtils {
       ApplicationAttemptId appAttemptId) throws IOException {
     store.applicationAttemptStarted(ApplicationAttemptStartData.newInstance(
       appAttemptId, appAttemptId.toString(), 0,
-      ContainerId.newInstance(appAttemptId, 1)));
+      ContainerId.newContainerId(appAttemptId, 1)));
   }
 
   protected void writeApplicationAttemptFinishData(

+ 3 - 3
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryClientService.java

@@ -142,7 +142,7 @@ public class TestApplicationHistoryClientService {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     GetContainerReportRequest request =
         GetContainerReportRequest.newInstance(containerId);
     GetContainerReportResponse response =
@@ -160,8 +160,8 @@ public class TestApplicationHistoryClientService {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
-    ContainerId containerId1 = ContainerId.newInstance(appAttemptId, 2);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
+    ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 2);
     GetContainersRequest request =
         GetContainersRequest.newInstance(appAttemptId);
     GetContainersResponse response =

+ 4 - 4
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestApplicationHistoryManagerOnTimelineStore.java

@@ -141,7 +141,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
         store.put(entities);
         for (int k = 1; k <= scale; ++k) {
           entities = new TimelineEntities();
-          ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
+          ContainerId containerId = ContainerId.newContainerId(appAttemptId, k);
           entities.addEntity(createContainerEntity(containerId));
           store.put(entities);
         }
@@ -238,7 +238,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     }
     Assert.assertNotNull(appAttempt);
     Assert.assertEquals(appAttemptId, appAttempt.getApplicationAttemptId());
-    Assert.assertEquals(ContainerId.newInstance(appAttemptId, 1),
+    Assert.assertEquals(ContainerId.newContainerId(appAttemptId, 1),
         appAttempt.getAMContainerId());
     Assert.assertEquals("test host", appAttempt.getHost());
     Assert.assertEquals(100, appAttempt.getRpcPort());
@@ -253,7 +253,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
   @Test
   public void testGetContainerReport() throws Exception {
     final ContainerId containerId =
-        ContainerId.newInstance(ApplicationAttemptId.newInstance(
+        ContainerId.newContainerId(ApplicationAttemptId.newInstance(
             ApplicationId.newInstance(0, 1), 1), 1);
     ContainerReport container;
     if (callerUGI == null) {
@@ -466,7 +466,7 @@ public class TestApplicationHistoryManagerOnTimelineStore {
     eventInfo.put(AppAttemptMetricsConstants.HOST_EVENT_INFO, "test host");
     eventInfo.put(AppAttemptMetricsConstants.RPC_PORT_EVENT_INFO, 100);
     eventInfo.put(AppAttemptMetricsConstants.MASTER_CONTAINER_EVENT_INFO,
-        ContainerId.newInstance(appAttemptId, 1));
+        ContainerId.newContainerId(appAttemptId, 1));
     tEvent.setEventInfo(eventInfo);
     entity.addEvent(tEvent);
     tEvent = new TimelineEvent();

+ 5 - 5
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java

@@ -121,7 +121,7 @@ public class TestFileSystemApplicationHistoryStore extends
         }
         // write container history data
         for (int k = 1; k <= num; ++k) {
-          ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
+          ContainerId containerId = ContainerId.newContainerId(appAttemptId, k);
           writeContainerStartData(containerId);
           if (missingContainer && k == num) {
             continue;
@@ -172,7 +172,7 @@ public class TestFileSystemApplicationHistoryStore extends
         // read container history data
         Assert.assertEquals(num, store.getContainers(appAttemptId).size());
         for (int k = 1; k <= num; ++k) {
-          ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
+          ContainerId containerId = ContainerId.newContainerId(appAttemptId, k);
           ContainerHistoryData containerData = store.getContainer(containerId);
           Assert.assertNotNull(containerData);
           Assert.assertEquals(Priority.newInstance(containerId.getId()),
@@ -187,7 +187,7 @@ public class TestFileSystemApplicationHistoryStore extends
         ContainerHistoryData masterContainer =
             store.getAMContainer(appAttemptId);
         Assert.assertNotNull(masterContainer);
-        Assert.assertEquals(ContainerId.newInstance(appAttemptId, 1),
+        Assert.assertEquals(ContainerId.newContainerId(appAttemptId, 1),
           masterContainer.getContainerId());
       }
     }
@@ -215,7 +215,7 @@ public class TestFileSystemApplicationHistoryStore extends
       Assert.assertTrue(e.getMessage().contains("is not opened"));
     }
     // write container history data
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     try {
       writeContainerStartData(containerId);
       Assert.fail();
@@ -240,7 +240,7 @@ public class TestFileSystemApplicationHistoryStore extends
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
     for (int i = 1; i <= 100000; ++i) {
-      ContainerId containerId = ContainerId.newInstance(appAttemptId, i);
+      ContainerId containerId = ContainerId.newContainerId(appAttemptId, i);
       writeContainerStartData(containerId);
       writeContainerFinishData(containerId);
     }

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestMemoryApplicationHistoryStore.java

@@ -137,7 +137,7 @@ public class TestMemoryApplicationHistoryStore extends
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     try {
       writeContainerFinishData(containerId);
       Assert.fail();
@@ -149,14 +149,14 @@ public class TestMemoryApplicationHistoryStore extends
     writeApplicationAttemptStartData(appAttemptId);
     int numContainers = 5;
     for (int i = 1; i <= numContainers; ++i) {
-      containerId = ContainerId.newInstance(appAttemptId, i);
+      containerId = ContainerId.newContainerId(appAttemptId, i);
       writeContainerStartData(containerId);
       writeContainerFinishData(containerId);
     }
     Assert
       .assertEquals(numContainers, store.getContainers(appAttemptId).size());
     for (int i = 1; i <= numContainers; ++i) {
-      containerId = ContainerId.newInstance(appAttemptId, i);
+      containerId = ContainerId.newContainerId(appAttemptId, i);
       ContainerHistoryData data = store.getContainer(containerId);
       Assert.assertNotNull(data);
       Assert.assertEquals(Priority.newInstance(containerId.getId()),
@@ -165,11 +165,11 @@ public class TestMemoryApplicationHistoryStore extends
     }
     ContainerHistoryData masterContainer = store.getAMContainer(appAttemptId);
     Assert.assertNotNull(masterContainer);
-    Assert.assertEquals(ContainerId.newInstance(appAttemptId, 1),
+    Assert.assertEquals(ContainerId.newContainerId(appAttemptId, 1),
       masterContainer.getContainerId());
     writeApplicationAttemptFinishData(appAttemptId);
     // Write again
-    containerId = ContainerId.newInstance(appAttemptId, 1);
+    containerId = ContainerId.newContainerId(appAttemptId, 1);
     try {
       writeContainerStartData(containerId);
       Assert.fail();
@@ -194,7 +194,7 @@ public class TestMemoryApplicationHistoryStore extends
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
     for (int i = 1; i <= numContainers; ++i) {
-      ContainerId containerId = ContainerId.newInstance(appAttemptId, i);
+      ContainerId containerId = ContainerId.newContainerId(appAttemptId, i);
       writeContainerStartData(containerId);
       writeContainerFinishData(containerId);
     }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebApp.java

@@ -134,7 +134,7 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
     containerPageInstance.set(
       YarnWebParams.CONTAINER_ID,
       ContainerId
-        .newInstance(
+        .newContainerId(
           ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1),
           1).toString());
     containerPageInstance.render();
@@ -153,7 +153,7 @@ public class TestAHSWebApp extends ApplicationHistoryStoreTestUtils {
             ApplicationAttemptId.newInstance(appId, j);
         writeApplicationAttemptStartData(appAttemptId);
         for (int k = 1; k <= numContainers; ++k) {
-          ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
+          ContainerId containerId = ContainerId.newContainerId(appAttemptId, k);
           writeContainerStartData(containerId);
           writeContainerFinishData(containerId);
         }

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java

@@ -338,7 +338,7 @@ public class TestAHSWebServices extends JerseyTest {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
     WebResource r = resource();
     ClientResponse response =
         r.path("ws").path("v1").path("applicationhistory").path("apps")

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

@@ -139,7 +139,7 @@ public class BuilderUtils {
 
   public static ContainerId newContainerId(ApplicationAttemptId appAttemptId,
       long containerId) {
-    return ContainerId.newInstance(appAttemptId, containerId);
+    return ContainerId.newContainerId(appAttemptId, containerId);
   }
 
   public static ContainerId newContainerId(int appId, int appAttemptId,
@@ -164,7 +164,7 @@ public class BuilderUtils {
   public static ContainerId newContainerId(RecordFactory recordFactory,
       ApplicationId appId, ApplicationAttemptId appAttemptId,
       int containerId) {
-    return ContainerId.newInstance(appAttemptId, containerId);
+    return ContainerId.newContainerId(appAttemptId, containerId);
   }
 
   public static NodeId newNodeId(String host, int port) {

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java

@@ -223,7 +223,7 @@ public class TestYarnServerApiClasses {
   }
 
   private ContainerId getContainerId(int containerID, int appAttemptId) {
-    ContainerId containerId = ContainerIdPBImpl.newInstance(
+    ContainerId containerId = ContainerIdPBImpl.newContainerId(
         getApplicationAttemptId(appAttemptId), containerID);
     return containerId;
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestProtocolRecords.java

@@ -51,7 +51,7 @@ public class TestProtocolRecords {
   public void testNMContainerStatus() {
     ApplicationId appId = ApplicationId.newInstance(123456789, 1);
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(attemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
     Resource resource = Resource.newInstance(1000, 200);
 
     NMContainerStatus report =
@@ -76,7 +76,7 @@ public class TestProtocolRecords {
   public void testRegisterNodeManagerRequest() {
     ApplicationId appId = ApplicationId.newInstance(123456789, 1);
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(attemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
 
     NMContainerStatus containerReport =
         NMContainerStatus.newInstance(containerId,

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/server/api/protocolrecords/TestRegisterNodeManagerRequest.java

@@ -38,7 +38,7 @@ public class TestRegisterNodeManagerRequest {
         RegisterNodeManagerRequest.newInstance(
           NodeId.newInstance("host", 1234), 1234, Resource.newInstance(0, 0),
           "version", Arrays.asList(NMContainerStatus.newInstance(
-            ContainerId.newInstance(
+            ContainerId.newContainerId(
               ApplicationAttemptId.newInstance(
                 ApplicationId.newInstance(1234L, 1), 1), 1),
             ContainerState.RUNNING, Resource.newInstance(1024, 1), "good", -1,

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

@@ -139,7 +139,7 @@ public class TestEventFlow {
     ApplicationId applicationId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 0);
-    ContainerId cID = ContainerId.newInstance(applicationAttemptId, 0);
+    ContainerId cID = ContainerId.newContainerId(applicationAttemptId, 0);
 
     String user = "testing";
     StartContainerRequest scRequest =

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

@@ -431,7 +431,7 @@ public class TestLinuxContainerExecutor {
     ApplicationId appId = ApplicationId.newInstance(12345, 67890);
     ApplicationAttemptId attemptId =
         ApplicationAttemptId.newInstance(appId, 54321);
-    ContainerId cid = ContainerId.newInstance(attemptId, 9876);
+    ContainerId cid = ContainerId.newContainerId(attemptId, 9876);
 
     Configuration conf = new YarnConfiguration();
     conf.setClass(YarnConfiguration.NM_LINUX_CONTAINER_RESOURCES_HANDLER,

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

@@ -290,7 +290,7 @@ public class TestNodeManagerReboot {
     ApplicationId appId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 0);
     return containerId;
   }
 

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

@@ -592,7 +592,7 @@ public class TestNodeManagerResync {
     ApplicationId applicationId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 1);
-    ContainerId containerId = ContainerId.newInstance(applicationAttemptId, id);
+    ContainerId containerId = ContainerId.newContainerId(applicationAttemptId, id);
     NMContainerStatus containerReport =
         NMContainerStatus.newInstance(containerId, containerState,
           Resource.newInstance(1024, 1), "recover container", 0,

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

@@ -260,7 +260,7 @@ public class TestNodeManagerShutdown {
     ApplicationId appId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 0);
     return containerId;
   }
   

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

@@ -224,7 +224,7 @@ public class TestNodeStatusUpdater {
         ApplicationAttemptId appAttemptID =
             ApplicationAttemptId.newInstance(appId1, 0);
         ContainerId firstContainerID =
-            ContainerId.newInstance(appAttemptID, heartBeatID);
+            ContainerId.newContainerId(appAttemptID, heartBeatID);
         ContainerLaunchContext launchContext = recordFactory
             .newRecordInstance(ContainerLaunchContext.class);
         Resource resource = BuilderUtils.newResource(2, 1);
@@ -254,7 +254,7 @@ public class TestNodeStatusUpdater {
         ApplicationAttemptId appAttemptID =
             ApplicationAttemptId.newInstance(appId2, 0);
         ContainerId secondContainerID =
-            ContainerId.newInstance(appAttemptID, heartBeatID);
+            ContainerId.newContainerId(appAttemptID, heartBeatID);
         ContainerLaunchContext launchContext = recordFactory
             .newRecordInstance(ContainerLaunchContext.class);
         long currentTime = System.currentTimeMillis();
@@ -818,7 +818,7 @@ public class TestNodeStatusUpdater {
     ApplicationId appId = ApplicationId.newInstance(0, 0);                    
     ApplicationAttemptId appAttemptId =                                       
         ApplicationAttemptId.newInstance(appId, 0);                           
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
     nm.getNMContext().getApplications().putIfAbsent(appId,
         mock(Application.class));
     nm.getNMContext().getContainers().putIfAbsent(cId, mock(Container.class));
@@ -855,7 +855,7 @@ public class TestNodeStatusUpdater {
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 0);
   
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 1);
     Token containerToken =
         BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
             BuilderUtils.newResource(1024, 1), 0, 123,
@@ -876,7 +876,7 @@ public class TestNodeStatusUpdater {
     };
 
     ContainerId runningContainerId =
-        ContainerId.newInstance(appAttemptId, 3);
+        ContainerId.newContainerId(appAttemptId, 3);
     Token runningContainerToken =
         BuilderUtils.newContainerToken(runningContainerId, "anyHost",
           1234, "anyUser", BuilderUtils.newResource(1024, 1), 0, 123,
@@ -936,7 +936,7 @@ public class TestNodeStatusUpdater {
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 0);
 
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 1);
     Token containerToken =
         BuilderUtils.newContainerToken(cId, "anyHost", 1234, "anyUser",
             BuilderUtils.newResource(1024, 1), 0, 123,
@@ -1494,7 +1494,7 @@ public class TestNodeStatusUpdater {
     ApplicationId applicationId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId applicationAttemptId =
         ApplicationAttemptId.newInstance(applicationId, 1);
-    ContainerId contaierId = ContainerId.newInstance(applicationAttemptId, id);
+    ContainerId contaierId = ContainerId.newContainerId(applicationAttemptId, id);
     ContainerStatus containerStatus =
         BuilderUtils.newContainerStatus(contaierId, containerState,
           "test_containerStatus: id=" + id + ", containerState: "

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

@@ -189,7 +189,7 @@ public class TestAuxServices {
 
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(appId1, 1);
     ContainerTokenIdentifier cti = new ContainerTokenIdentifier(
-        ContainerId.newInstance(attemptId, 1), "", "",
+        ContainerId.newContainerId(attemptId, 1), "", "",
         Resource.newInstance(1, 1), 0,0,0, Priority.newInstance(0), 0);
     Container container = new ContainerImpl(null, null, null, null, null,
         null, cti);

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

@@ -104,7 +104,7 @@ public class TestContainerManager extends BaseContainerManagerTest {
     ApplicationId appId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, id);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
     return containerId;
   }
   

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

@@ -111,7 +111,7 @@ public class TestContainerManagerRecovery {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationAttemptId attemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId cid = ContainerId.newInstance(attemptId, 1);
+    ContainerId cid = ContainerId.newContainerId(attemptId, 1);
     Map<String, LocalResource> localResources = Collections.emptyMap();
     Map<String, String> containerEnv = Collections.emptyMap();
     List<String> containerCmds = Collections.emptyList();

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

@@ -385,7 +385,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
 
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
     Map<String, String> userSetEnv = new HashMap<String, String>();
     userSetEnv.put(Environment.CONTAINER_ID.name(), "user_set_container_id");
     userSetEnv.put(Environment.NM_HOST.name(), "user_set_NM_HOST");
@@ -634,7 +634,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
     File processStartFile =
         new File(tmpDir, "pid.txt").getAbsoluteFile();
 
@@ -771,7 +771,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
   @Test (timeout = 10000)
   public void testCallFailureWithNullLocalizedResources() {
     Container container = mock(Container.class);
-    when(container.getContainerId()).thenReturn(ContainerId.newInstance(
+    when(container.getContainerId()).thenReturn(ContainerId.newContainerId(
         ApplicationAttemptId.newInstance(ApplicationId.newInstance(
             System.currentTimeMillis(), 1), 1), 1));
     ContainerLaunchContext clc = mock(ContainerLaunchContext.class);
@@ -980,7 +980,7 @@ public class TestContainerLaunch extends BaseContainerManagerTest {
     ApplicationId appId = ApplicationId.newInstance(2, 2);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
     File processStartFile =
         new File(tmpDir, "pid.txt").getAbsoluteFile();
     File childProcessStartFile =

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

@@ -206,7 +206,7 @@ public class TestContainersMonitor extends BaseContainerManagerTest {
     // ////// Construct the Container-id
     ApplicationId appId = ApplicationId.newInstance(0, 0);
     ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
     int port = 12345;
 
     URL resource_alpha =

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/recovery/TestNMLeveldbStateStoreService.java

@@ -226,7 +226,7 @@ public class TestNMLeveldbStateStoreService {
     ApplicationId appId = ApplicationId.newInstance(1234, 3);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 4);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 5);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 5);
     LocalResource lrsrc = LocalResource.newInstance(
         URL.newInstance("hdfs", "somehost", 12345, "/some/path/to/rsrc"),
         LocalResourceType.FILE, LocalResourceVisibility.APPLICATION, 123L,

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

@@ -378,7 +378,7 @@ public class TestApplicationCleanup {
     // nm1/nm2 register to rm2, and do a heartbeat
     nm1.setResourceTrackerService(rm2.getResourceTrackerService());
     nm1.registerNode(Arrays.asList(NMContainerStatus.newInstance(
-      ContainerId.newInstance(am0.getApplicationAttemptId(), 1),
+      ContainerId.newContainerId(am0.getApplicationAttemptId(), 1),
       ContainerState.COMPLETE, Resource.newInstance(1024, 1), "", 0,
       Priority.newInstance(0), 1234)), Arrays.asList(app0.getApplicationId()));
     nm2.setResourceTrackerService(rm2.getResourceTrackerService());

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

@@ -404,7 +404,7 @@ public class TestClientRMService {
         .newRecordInstance(GetContainerReportRequest.class);
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
         ApplicationId.newInstance(123456, 1), 1);
-    ContainerId containerId = ContainerId.newInstance(attemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
     request.setContainerId(containerId);
 
     try {
@@ -425,7 +425,7 @@ public class TestClientRMService {
         .newRecordInstance(GetContainersRequest.class);
     ApplicationAttemptId attemptId = ApplicationAttemptId.newInstance(
         ApplicationId.newInstance(123456, 1), 1);
-    ContainerId containerId = ContainerId.newInstance(attemptId, 1);
+    ContainerId containerId = ContainerId.newContainerId(attemptId, 1);
     request.setApplicationAttemptId(attemptId);
     try {
       GetContainersResponse response = rmService.getContainers(request);
@@ -1213,7 +1213,7 @@ public class TestClientRMService {
     RMAppAttemptImpl rmAppAttemptImpl = spy(new RMAppAttemptImpl(attemptId,
         rmContext, yarnScheduler, null, asContext, config, false, null));
     Container container = Container.newInstance(
-        ContainerId.newInstance(attemptId, 1), null, "", null, null, null);
+        ContainerId.newContainerId(attemptId, 1), null, "", null, null, null);
     RMContainerImpl containerimpl = spy(new RMContainerImpl(container,
         attemptId, null, "", rmContext));
     Map<ApplicationAttemptId, RMAppAttempt> attempts = 

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java

@@ -207,7 +207,7 @@ public class TestContainerResourceUsage {
     // usage metrics. This will cause the attempt to fail, and, since the max
     // attempt retries is 1, the app will also fail. This is intentional so
     // that all containers will complete prior to saving.
-    ContainerId cId = ContainerId.newInstance(attempt0.getAppAttemptId(), 1);
+    ContainerId cId = ContainerId.newContainerId(attempt0.getAppAttemptId(), 1);
     nm.nodeHeartbeat(attempt0.getAppAttemptId(),
                  cId.getContainerId(), ContainerState.COMPLETE);
     rm0.waitForState(nm, cId, RMContainerState.COMPLETED);
@@ -289,7 +289,7 @@ public class TestContainerResourceUsage {
 
     // launch the 2nd container.
     ContainerId containerId2 =
-        ContainerId.newInstance(am0.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am0.getApplicationAttemptId(), 2);
     nm.nodeHeartbeat(am0.getApplicationAttemptId(),
                       containerId2.getContainerId(), ContainerState.RUNNING);
     rm.waitForState(nm, containerId2, RMContainerState.RUNNING);

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

@@ -1963,7 +1963,7 @@ public class TestRMRestart extends ParameterizedSchedulerTestBase {
 
   public static NMContainerStatus createNMContainerStatus(
       ApplicationAttemptId appAttemptId, int id, ContainerState containerState) {
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, id);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, id);
     NMContainerStatus containerReport =
         NMContainerStatus.newInstance(containerId, containerState,
           Resource.newInstance(1024, 1), "recover container", 0,

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

@@ -510,7 +510,7 @@ public class TestResourceTrackerService {
     // Case 1.1: AppAttemptId is null
     NMContainerStatus report =
         NMContainerStatus.newInstance(
-          ContainerId.newInstance(
+          ContainerId.newContainerId(
             ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1),
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
@@ -522,7 +522,7 @@ public class TestResourceTrackerService {
         (RMAppAttemptImpl) app.getCurrentAppAttempt();
     currentAttempt.setMasterContainer(null);
     report = NMContainerStatus.newInstance(
-          ContainerId.newInstance(currentAttempt.getAppAttemptId(), 0),
+          ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0),
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
     rm.getResourceTrackerService().handleNMContainerStatus(report, null);
@@ -533,7 +533,7 @@ public class TestResourceTrackerService {
 
     // Case 2.1: AppAttemptId is null
     report = NMContainerStatus.newInstance(
-          ContainerId.newInstance(
+          ContainerId.newContainerId(
             ApplicationAttemptId.newInstance(app.getApplicationId(), 2), 1),
           ContainerState.COMPLETE, Resource.newInstance(1024, 1),
           "Dummy Completed", 0, Priority.newInstance(10), 1234);
@@ -549,7 +549,7 @@ public class TestResourceTrackerService {
         (RMAppAttemptImpl) app.getCurrentAppAttempt();
     currentAttempt.setMasterContainer(null);
     report = NMContainerStatus.newInstance(
-      ContainerId.newInstance(currentAttempt.getAppAttemptId(), 0),
+      ContainerId.newContainerId(currentAttempt.getAppAttemptId(), 0),
       ContainerState.COMPLETE, Resource.newInstance(1024, 1),
       "Dummy Completed", 0, Priority.newInstance(10), 1234);
     try {

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

@@ -849,7 +849,7 @@ public class TestWorkPreservingRMRestart {
 
     // try to release a container before the container is actually recovered.
     final ContainerId runningContainer =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     am1.allocate(null, Arrays.asList(runningContainer));
 
     // send container statuses to recover the containers

+ 6 - 6
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/ahs/TestRMApplicationHistoryWriter.java

@@ -153,7 +153,7 @@ public class TestRMApplicationHistoryWriter {
     when(appAttempt.getRpcPort()).thenReturn(-100);
     Container container = mock(Container.class);
     when(container.getId())
-      .thenReturn(ContainerId.newInstance(appAttemptId, 1));
+      .thenReturn(ContainerId.newContainerId(appAttemptId, 1));
     when(appAttempt.getMasterContainer()).thenReturn(container);
     when(appAttempt.getDiagnostics()).thenReturn("test diagnostics info");
     when(appAttempt.getTrackingUrl()).thenReturn("test url");
@@ -254,7 +254,7 @@ public class TestRMApplicationHistoryWriter {
     Assert.assertNotNull(appAttemptHD);
     Assert.assertEquals("test host", appAttemptHD.getHost());
     Assert.assertEquals(-100, appAttemptHD.getRPCPort());
-    Assert.assertEquals(ContainerId.newInstance(
+    Assert.assertEquals(ContainerId.newContainerId(
       ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1), 1),
       appAttemptHD.getMasterContainerId());
 
@@ -281,14 +281,14 @@ public class TestRMApplicationHistoryWriter {
   @Test
   public void testWriteContainer() throws Exception {
     RMContainer container =
-        createRMContainer(ContainerId.newInstance(
+        createRMContainer(ContainerId.newContainerId(
           ApplicationAttemptId.newInstance(ApplicationId.newInstance(0, 1), 1),
           1));
     writer.containerStarted(container);
     ContainerHistoryData containerHD = null;
     for (int i = 0; i < MAX_RETRIES; ++i) {
       containerHD =
-          store.getContainer(ContainerId.newInstance(ApplicationAttemptId
+          store.getContainer(ContainerId.newContainerId(ApplicationAttemptId
             .newInstance(ApplicationId.newInstance(0, 1), 1), 1));
       if (containerHD != null) {
         break;
@@ -307,7 +307,7 @@ public class TestRMApplicationHistoryWriter {
     writer.containerFinished(container);
     for (int i = 0; i < MAX_RETRIES; ++i) {
       containerHD =
-          store.getContainer(ContainerId.newInstance(ApplicationAttemptId
+          store.getContainer(ContainerId.newContainerId(ApplicationAttemptId
             .newInstance(ApplicationId.newInstance(0, 1), 1), 1));
       if (containerHD.getContainerState() != null) {
         break;
@@ -337,7 +337,7 @@ public class TestRMApplicationHistoryWriter {
         RMAppAttempt appAttempt = createRMAppAttempt(appAttemptId);
         writer.applicationAttemptStarted(appAttempt);
         for (int k = 1; k <= 10; ++k) {
-          ContainerId containerId = ContainerId.newInstance(appAttemptId, k);
+          ContainerId containerId = ContainerId.newContainerId(appAttemptId, k);
           RMContainer container = createRMContainer(containerId);
           writer.containerStarted(container);
           writer.containerFinished(container);

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

@@ -189,7 +189,7 @@ public abstract class MockAsm extends MockApps {
     final ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(newAppID(i), 0);
     final Container masterContainer = Records.newRecord(Container.class);
-    ContainerId containerId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 0);
     masterContainer.setId(containerId);
     masterContainer.setNodeHttpAddress("node:port");
     final String user = newUserName();

+ 11 - 11
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/applicationsmanager/TestAMRestart.java

@@ -101,20 +101,20 @@ public class TestAMRestart {
     // launch the 2nd container, for testing running container transferred.
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
     ContainerId containerId2 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
 
     // launch the 3rd container, for testing container allocated by previous
     // attempt is completed by the next new attempt/
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING);
     ContainerId containerId3 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
     rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
 
     // 4th container still in AQUIRED state. for testing Acquired container is
     // always killed.
     ContainerId containerId4 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 4);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 4);
     rm1.waitForState(nm1, containerId4, RMContainerState.ACQUIRED);
 
     // 5th container is in Allocated state. for testing allocated container is
@@ -122,14 +122,14 @@ public class TestAMRestart {
     am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
     nm1.nodeHeartbeat(true);
     ContainerId containerId5 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 5);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 5);
     rm1.waitForContainerAllocated(nm1, containerId5);
     rm1.waitForState(nm1, containerId5, RMContainerState.ALLOCATED);
 
     // 6th container is in Reserved state.
     am1.allocate("127.0.0.1", 6000, 1, new ArrayList<ContainerId>());
     ContainerId containerId6 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 6);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 6);
     nm1.nodeHeartbeat(true);
     SchedulerApplicationAttempt schedulerAttempt =
         ((AbstractYarnScheduler) rm1.getResourceScheduler())
@@ -295,12 +295,12 @@ public class TestAMRestart {
     // launch the container-2
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 2, ContainerState.RUNNING);
     ContainerId containerId2 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, containerId2, RMContainerState.RUNNING);
     // launch the container-3
     nm1.nodeHeartbeat(am1.getApplicationAttemptId(), 3, ContainerState.RUNNING);
     ContainerId containerId3 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
     rm1.waitForState(nm1, containerId3, RMContainerState.RUNNING);
     
     // fail am1
@@ -335,7 +335,7 @@ public class TestAMRestart {
     }
     nm1.nodeHeartbeat(am2.getApplicationAttemptId(), 2, ContainerState.RUNNING);
     ContainerId am2ContainerId2 =
-        ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, am2ContainerId2, RMContainerState.RUNNING);
 
     // fail am2.
@@ -379,7 +379,7 @@ public class TestAMRestart {
     CapacityScheduler scheduler =
         (CapacityScheduler) rm1.getResourceScheduler();
     ContainerId amContainer =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 1);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
     // Preempt the first attempt;
     scheduler.killContainer(scheduler.getRMContainer(amContainer));
 
@@ -396,7 +396,7 @@ public class TestAMRestart {
 
     // Preempt the second attempt.
     ContainerId amContainer2 =
-        ContainerId.newInstance(am2.getApplicationAttemptId(), 1);
+        ContainerId.newContainerId(am2.getApplicationAttemptId(), 1);
     scheduler.killContainer(scheduler.getRMContainer(amContainer2));
 
     am2.waitForState(RMAppAttemptState.FAILED);
@@ -487,7 +487,7 @@ public class TestAMRestart {
     CapacityScheduler scheduler =
         (CapacityScheduler) rm1.getResourceScheduler();
     ContainerId amContainer =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 1);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 1);
 
     // Forcibly preempt the am container;
     scheduler.killContainer(scheduler.getRMContainer(amContainer));

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/metrics/TestSystemMetricsPublisher.java

@@ -250,7 +250,7 @@ public class TestSystemMetricsPublisher {
   @Test(timeout = 10000)
   public void testPublishContainerMetrics() throws Exception {
     ContainerId containerId =
-        ContainerId.newInstance(ApplicationAttemptId.newInstance(
+        ContainerId.newContainerId(ApplicationAttemptId.newInstance(
             ApplicationId.newInstance(0, 1), 1), 1);
     RMContainer container = createRMContainer(containerId);
     metricsPublisher.containerCreated(container, container.getCreationTime());
@@ -347,7 +347,7 @@ public class TestSystemMetricsPublisher {
     when(appAttempt.getRpcPort()).thenReturn(-100);
     Container container = mock(Container.class);
     when(container.getId())
-        .thenReturn(ContainerId.newInstance(appAttemptId, 1));
+        .thenReturn(ContainerId.newContainerId(appAttemptId, 1));
     when(appAttempt.getMasterContainer()).thenReturn(container);
     when(appAttempt.getDiagnostics()).thenReturn("test diagnostics info");
     when(appAttempt.getTrackingUrl()).thenReturn("test tracking url");

+ 1 - 1
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/monitor/capacity/TestProportionalCapacityPreemptionPolicy.java

@@ -728,7 +728,7 @@ public class TestProportionalCapacityPreemptionPolicy {
 
   RMContainer mockContainer(ApplicationAttemptId appAttId, int id,
       Resource r, int priority) {
-    ContainerId cId = ContainerId.newInstance(appAttId, id);
+    ContainerId cId = ContainerId.newContainerId(appAttId, id);
     Container c = mock(Container.class);
     when(c.getResource()).thenReturn(r);
     when(c.getPriority()).thenReturn(Priority.create(priority));

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

@@ -1395,7 +1395,7 @@ public class TestRMAppAttemptTransitions {
     // failed attempt captured the container finished event.
     assertEquals(0, applicationAttempt.getJustFinishedContainers().size());
     ContainerStatus cs2 =
-        ContainerStatus.newInstance(ContainerId.newInstance(appAttemptId, 2),
+        ContainerStatus.newInstance(ContainerId.newContainerId(appAttemptId, 2),
           ContainerState.COMPLETE, "", 0);
     applicationAttempt.handle(new RMAppAttemptContainerFinishedEvent(
       appAttemptId, cs2, anyNodeId));

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

@@ -249,7 +249,7 @@ public class TestRMContainerImpl {
 
     // request a container.
     am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
-    ContainerId containerId2 = ContainerId.newInstance(
+    ContainerId containerId2 = ContainerId.newContainerId(
         am1.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
 

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

@@ -138,7 +138,7 @@ public class TestSchedulerApplicationAttempt {
   
   private RMContainer createRMContainer(ApplicationAttemptId appAttId, int id,
       Resource resource) {
-    ContainerId containerId = ContainerId.newInstance(appAttId, id);
+    ContainerId containerId = ContainerId.newContainerId(appAttId, id);
     RMContainer rmContainer = mock(RMContainer.class);
     Container container = mock(Container.class);
     when(container.getResource()).thenReturn(resource);

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/TestSchedulerUtils.java

@@ -560,7 +560,7 @@ public class TestSchedulerUtils {
   @Test
   public void testCreateAbnormalContainerStatus() {
     ContainerStatus cd = SchedulerUtils.createAbnormalContainerStatus(
-        ContainerId.newInstance(ApplicationAttemptId.newInstance(
+        ContainerId.newContainerId(ApplicationAttemptId.newInstance(
           ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
     Assert.assertEquals(ContainerExitStatus.ABORTED, cd.getExitStatus());
   }
@@ -568,7 +568,7 @@ public class TestSchedulerUtils {
   @Test
   public void testCreatePreemptedContainerStatus() {
     ContainerStatus cd = SchedulerUtils.createPreemptedContainerStatus(
-        ContainerId.newInstance(ApplicationAttemptId.newInstance(
+        ContainerId.newContainerId(ApplicationAttemptId.newInstance(
           ApplicationId.newInstance(System.currentTimeMillis(), 1), 1), 1), "x");
     Assert.assertEquals(ContainerExitStatus.PREEMPTED, cd.getExitStatus());
   }

+ 2 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestCapacityScheduler.java

@@ -1085,7 +1085,7 @@ public class TestCapacityScheduler {
 
     // request a container.
     am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
-    ContainerId containerId1 = ContainerId.newInstance(
+    ContainerId containerId1 = ContainerId.newContainerId(
         am1.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, containerId1, RMContainerState.ALLOCATED);
 
@@ -1122,7 +1122,7 @@ public class TestCapacityScheduler {
     }
 
     // New container will be allocated and will move to ALLOCATED state
-    ContainerId containerId2 = ContainerId.newInstance(
+    ContainerId containerId2 = ContainerId.newContainerId(
         am1.getApplicationAttemptId(), 3);
     rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
 

+ 16 - 16
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestContainerAllocation.java

@@ -164,7 +164,7 @@ public class TestContainerAllocation {
     // request a container.
     am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
     ContainerId containerId2 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
 
     RMContainer container =
@@ -194,7 +194,7 @@ public class TestContainerAllocation {
     // request a container.
     am1.allocate("127.0.0.1", 1024, 1, new ArrayList<ContainerId>());
     ContainerId containerId2 =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, containerId2, RMContainerState.ALLOCATED);
 
     // acquire the container.
@@ -247,7 +247,7 @@ public class TestContainerAllocation {
     // request a container.
     am2.allocate("127.0.0.1", 512, 1, new ArrayList<ContainerId>());
     ContainerId containerId =
-        ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
     rm1.waitForState(nm1, containerId, RMContainerState.ALLOCATED);
 
     // acquire the container.
@@ -480,13 +480,13 @@ public class TestContainerAllocation {
     
     // A has only 10% of x, so it can only allocate one container in label=empty
     ContainerId containerId =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
     Assert.assertTrue(rm1.waitForState(nm3, containerId,
           RMContainerState.ALLOCATED, 10 * 1000));
     // Cannot allocate 2nd label=empty container
     containerId =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 3);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 3);
     am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "");
     Assert.assertFalse(rm1.waitForState(nm3, containerId,
           RMContainerState.ALLOCATED, 10 * 1000));
@@ -495,7 +495,7 @@ public class TestContainerAllocation {
     // We can allocate floor(8000 / 1024) = 7 containers
     for (int id = 3; id <= 8; id++) {
       containerId =
-          ContainerId.newInstance(am1.getApplicationAttemptId(), id);
+          ContainerId.newContainerId(am1.getApplicationAttemptId(), id);
       am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
       Assert.assertTrue(rm1.waitForState(nm1, containerId,
           RMContainerState.ALLOCATED, 10 * 1000));
@@ -571,7 +571,7 @@ public class TestContainerAllocation {
     // request a container (label = x && y). can only allocate on nm2 
     am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x && y");
     containerId =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm1, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm2, containerId,
@@ -588,7 +588,7 @@ public class TestContainerAllocation {
     // and now b1's queue capacity will be used, cannot allocate more containers
     // (Maximum capacity reached)
     am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm4, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertFalse(rm1.waitForState(nm5, containerId,
@@ -601,7 +601,7 @@ public class TestContainerAllocation {
     // request a container. try to allocate on nm1 (label = x) and nm3 (label =
     // y,z). Will successfully allocate on nm3
     am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
-    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm1, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm3, containerId,
@@ -612,7 +612,7 @@ public class TestContainerAllocation {
     // try to allocate container (request label = y && z) on nm3 (label = y) and
     // nm4 (label = y,z). Will sucessfully allocate on nm4 only.
     am3.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y && z");
-    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 3);
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 3);
     Assert.assertFalse(rm1.waitForState(nm3, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm4, containerId,
@@ -654,7 +654,7 @@ public class TestContainerAllocation {
     // request a container.
     am1.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "x");
     containerId =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm2, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm1, containerId,
@@ -669,7 +669,7 @@ public class TestContainerAllocation {
 
     // request a container.
     am2.allocate("*", 1024, 1, new ArrayList<ContainerId>(), "y");
-    containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm1, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm2, containerId,
@@ -684,7 +684,7 @@ public class TestContainerAllocation {
 
     // request a container.
     am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm2, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm3, containerId,
@@ -730,7 +730,7 @@ public class TestContainerAllocation {
     // request a container.
     am1.allocate("*", 1024, 1, new ArrayList<ContainerId>());
     containerId =
-        ContainerId.newInstance(am1.getApplicationAttemptId(), 2);
+        ContainerId.newContainerId(am1.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm3, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm1, containerId,
@@ -745,7 +745,7 @@ public class TestContainerAllocation {
 
     // request a container.
     am2.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newInstance(am2.getApplicationAttemptId(), 2);
+    containerId = ContainerId.newContainerId(am2.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm3, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm2, containerId,
@@ -760,7 +760,7 @@ public class TestContainerAllocation {
 
     // request a container.
     am3.allocate("*", 1024, 1, new ArrayList<ContainerId>());
-    containerId = ContainerId.newInstance(am3.getApplicationAttemptId(), 2);
+    containerId = ContainerId.newContainerId(am3.getApplicationAttemptId(), 2);
     Assert.assertFalse(rm1.waitForState(nm2, containerId,
         RMContainerState.ALLOCATED, 10 * 1000));
     Assert.assertTrue(rm1.waitForState(nm3, containerId,

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

@@ -3530,7 +3530,7 @@ public class TestFairScheduler extends FairSchedulerTestBase {
     // ResourceRequest will be empty once NodeUpdate is completed
     Assert.assertNull(app.getResourceRequest(priority, host));
 
-    ContainerId containerId1 = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId containerId1 = ContainerId.newContainerId(appAttemptId, 1);
     RMContainer rmContainer = app.getRMContainer(containerId1);
 
     // Create a preempt event and register for preemption

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

@@ -231,7 +231,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
         ApplicationAttemptId.newInstance(appId, 1);
     
     ContainerId validContainerId =
-        ContainerId.newInstance(validAppAttemptId, 0);
+        ContainerId.newContainerId(validAppAttemptId, 0);
     
     NodeId validNode = yarnCluster.getNodeManager(0).getNMContext().getNodeId();
     NodeId invalidNode = NodeId.newInstance("InvalidHost", 1234);
@@ -311,7 +311,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
         ApplicationAttemptId.newInstance(appId, 2);
         
     ContainerId validContainerId2 =
-        ContainerId.newInstance(validAppAttemptId2, 0);
+        ContainerId.newContainerId(validAppAttemptId2, 0);
 
     org.apache.hadoop.yarn.api.records.Token validContainerToken2 =
         containerTokenSecretManager.createContainerToken(validContainerId2,
@@ -401,7 +401,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
           .createNMToken(validAppAttemptId, validNode, user);
     org.apache.hadoop.yarn.api.records.Token newContainerToken =
         containerTokenSecretManager.createContainerToken(
-          ContainerId.newInstance(attempt2, 1), validNode, user, r,
+          ContainerId.newContainerId(attempt2, 1), validNode, user, r,
             Priority.newInstance(0), 0);
     Assert.assertTrue(testStartContainer(rpc, attempt2, validNode,
       newContainerToken, attempt1NMToken, false).isEmpty());
@@ -638,7 +638,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 0);
-    ContainerId cId = ContainerId.newInstance(appAttemptId, 0);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, 0);
     NodeManager nm = yarnCluster.getNodeManager(0);
     NMTokenSecretManagerInNM nmTokenSecretManagerInNM =
         nm.getNMContext().getNMTokenSecretManager();
@@ -691,7 +691,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     } while (containerTokenSecretManager.getCurrentKey().getKeyId()
         == tamperedContainerTokenSecretManager.getCurrentKey().getKeyId());
     
-    ContainerId cId2 = ContainerId.newInstance(appAttemptId, 1);
+    ContainerId cId2 = ContainerId.newContainerId(appAttemptId, 1);
     // Creating modified containerToken
     Token containerToken2 =
         tamperedContainerTokenSecretManager.createContainerToken(cId2, nodeId,
@@ -733,7 +733,7 @@ public class TestContainerManagerSecurity extends KerberosSecurityTestcase {
     ApplicationId appId = ApplicationId.newInstance(1, 1);
     ApplicationAttemptId appAttemptId =
         ApplicationAttemptId.newInstance(appId, 0);
-    ContainerId cId = ContainerId.newInstance(appAttemptId, (5L << 40) | 3L);
+    ContainerId cId = ContainerId.newContainerId(appAttemptId, (5L << 40) | 3L);
     NodeManager nm = yarnCluster.getNodeManager(0);
     NMTokenSecretManagerInNM nmTokenSecretManagerInNM =
         nm.getNMContext().getNMTokenSecretManager();