Forráskód Böngészése

YARN-2312. Deprecated old ContainerId#getId API and updated MapReduce to use ContainerId#getContainerId instead. Contributed by Tsuyoshi OZAWA

Jian He 10 éve
szülő
commit
0af1a2b5bc
25 módosított fájl, 124 hozzáadás és 72 törlés
  1. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java
  2. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/WrappedJvmID.java
  3. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java
  4. 2 2
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TaskAttemptImpl.java
  5. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/local/LocalContainerAllocator.java
  6. 4 3
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRAppBenchmark.java
  7. 1 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestCheckpointPreemptionPolicy.java
  8. 55 28
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JVMId.java
  9. 2 1
      hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapreduce/v2/TestMRJobs.java
  10. 3 0
      hadoop-yarn-project/CHANGES.txt
  11. 4 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-api/src/main/java/org/apache/hadoop/yarn/api/records/ContainerId.java
  12. 6 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-client/src/test/java/org/apache/hadoop/yarn/client/api/async/impl/TestNMClientAsync.java
  13. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java
  14. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/ApplicationHistoryStoreTestUtils.java
  15. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestFileSystemApplicationHistoryStore.java
  16. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/TestMemoryApplicationHistoryStore.java
  17. 1 0
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-applicationhistoryservice/src/test/java/org/apache/hadoop/yarn/server/applicationhistoryservice/webapp/TestAHSWebServices.java
  18. 2 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-common/src/test/java/org/apache/hadoop/yarn/TestYarnServerApiClasses.java
  19. 4 2
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java
  20. 6 6
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/TestContainerManager.java
  21. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/test/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/localizer/TestLocalizedResource.java
  22. 1 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/MockNM.java
  23. 7 7
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestContainerResourceUsage.java
  24. 6 3
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/TestRM.java
  25. 2 1
      hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-resourcemanager/src/test/java/org/apache/hadoop/yarn/server/resourcemanager/scheduler/capacity/TestUtils.java

+ 1 - 1
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/MapReduceChildJVM.java

@@ -157,7 +157,7 @@ public class MapReduceChildJVM {
 
 
   public static List<String> getVMCommand(
   public static List<String> getVMCommand(
       InetSocketAddress taskAttemptListenerAddr, Task task, 
       InetSocketAddress taskAttemptListenerAddr, Task task, 
-      ID jvmID) {
+      JVMId jvmID) {
 
 
     TaskAttemptID attemptID = task.getTaskID();
     TaskAttemptID attemptID = task.getTaskID();
     JobConf conf = task.conf;
     JobConf conf = task.conf;

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/WrappedJvmID.java

@@ -23,8 +23,8 @@ package org.apache.hadoop.mapred;
  */
  */
 public class WrappedJvmID extends JVMId {
 public class WrappedJvmID extends JVMId {
 
 
-  public WrappedJvmID(JobID jobID, boolean mapTask, int nextInt) {
-    super(jobID, mapTask, nextInt);
+  public WrappedJvmID(JobID jobID, boolean mapTask, long nextLong) {
+    super(jobID, mapTask, nextLong);
   }
   }
 
 
 }
 }

+ 2 - 2
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapred/YarnChild.java

@@ -82,9 +82,9 @@ class YarnChild {
     final InetSocketAddress address =
     final InetSocketAddress address =
         NetUtils.createSocketAddrForHost(host, port);
         NetUtils.createSocketAddrForHost(host, port);
     final TaskAttemptID firstTaskid = TaskAttemptID.forName(args[2]);
     final TaskAttemptID firstTaskid = TaskAttemptID.forName(args[2]);
-    int jvmIdInt = Integer.parseInt(args[3]);
+    long jvmIdLong = Long.parseLong(args[3]);
     JVMId jvmId = new JVMId(firstTaskid.getJobID(),
     JVMId jvmId = new JVMId(firstTaskid.getJobID(),
-        firstTaskid.getTaskType() == TaskType.MAP, jvmIdInt);
+        firstTaskid.getTaskType() == TaskType.MAP, jvmIdLong);
 
 
     // initialize metrics
     // initialize metrics
     DefaultMetricsSystem.initialize(
     DefaultMetricsSystem.initialize(

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

@@ -1517,8 +1517,8 @@ public abstract class TaskAttemptImpl implements
       taskAttempt.remoteTask = taskAttempt.createRemoteTask();
       taskAttempt.remoteTask = taskAttempt.createRemoteTask();
       taskAttempt.jvmID =
       taskAttempt.jvmID =
           new WrappedJvmID(taskAttempt.remoteTask.getTaskID().getJobID(),
           new WrappedJvmID(taskAttempt.remoteTask.getTaskID().getJobID(),
-            taskAttempt.remoteTask.isMapTask(), taskAttempt.container.getId()
-              .getId());
+              taskAttempt.remoteTask.isMapTask(),
+              taskAttempt.container.getId().getContainerId());
       taskAttempt.taskAttemptListener.registerPendingTask(
       taskAttempt.taskAttemptListener.registerPendingTask(
           taskAttempt.remoteTask, taskAttempt.jvmID);
           taskAttempt.remoteTask, taskAttempt.jvmID);
 
 

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

@@ -150,7 +150,7 @@ public class LocalContainerAllocator extends RMCommunicator
       // Assign the same container ID as the AM
       // Assign the same container ID as the AM
       ContainerId cID =
       ContainerId cID =
           ContainerId.newInstance(getContext().getApplicationAttemptId(),
           ContainerId.newInstance(getContext().getApplicationAttemptId(),
-            this.containerId.getId());
+            this.containerId.getContainerId());
       Container container = recordFactory.newRecordInstance(Container.class);
       Container container = recordFactory.newRecordInstance(Container.class);
       container.setId(cID);
       container.setId(cID);
       NodeId nodeId = NodeId.newInstance(this.nmHost, this.nmPort);
       NodeId nodeId = NodeId.newInstance(this.nmHost, this.nmPort);

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

@@ -244,9 +244,10 @@ public class MRAppBenchmark {
                           getContext().getApplicationAttemptId(),
                           getContext().getApplicationAttemptId(),
                           request.getResponseId() + i);
                           request.getResponseId() + i);
                     containers.add(Container.newInstance(containerId,
                     containers.add(Container.newInstance(containerId,
-                      NodeId.newInstance("host" + containerId.getId(), 2345),
-                      "host" + containerId.getId() + ":5678",
-                      req.getCapability(), req.getPriority(), null));
+                        NodeId.newInstance(
+                            "host" + containerId.getContainerId(), 2345),
+                        "host" + containerId.getContainerId() + ":5678",
+                        req.getCapability(), req.getPriority(), null));
                   }
                   }
                 }
                 }
 
 

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

@@ -101,7 +101,7 @@ public class TestCheckpointPreemptionPolicy {
 
 
     for (Map.Entry<ContainerId,TaskAttemptId> ent :
     for (Map.Entry<ContainerId,TaskAttemptId> ent :
          assignedContainers.entrySet()) {
          assignedContainers.entrySet()) {
-      System.out.println("cont:" + ent.getKey().getId() +
+      System.out.println("cont:" + ent.getKey().getContainerId() +
           " type:" + ent.getValue().getTaskId().getTaskType() +
           " type:" + ent.getValue().getTaskId().getTaskType() +
           " res:" + contToResourceMap.get(ent.getKey()).getMemory() + "MB" );
           " res:" + contToResourceMap.get(ent.getKey()).getMemory() + "MB" );
     }
     }

+ 55 - 28
hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/JVMId.java

@@ -23,23 +23,25 @@ import java.io.DataOutput;
 import java.io.IOException;
 import java.io.IOException;
 import java.text.NumberFormat;
 import java.text.NumberFormat;
 
 
-class JVMId extends ID {
+class JVMId {
   boolean isMap;
   boolean isMap;
-  JobID jobId;
+  final JobID jobId;
+  private long jvmId;
   private static final String JVM = "jvm";
   private static final String JVM = "jvm";
+  private static final char SEPARATOR = '_';
   private static NumberFormat idFormat = NumberFormat.getInstance();
   private static NumberFormat idFormat = NumberFormat.getInstance();
   static {
   static {
     idFormat.setGroupingUsed(false);
     idFormat.setGroupingUsed(false);
     idFormat.setMinimumIntegerDigits(6);
     idFormat.setMinimumIntegerDigits(6);
   }
   }
   
   
-  public JVMId(JobID jobId, boolean isMap, int id) {
-    super(id);
+  public JVMId(JobID jobId, boolean isMap, long id) {
+    this.jvmId = id;
     this.isMap = isMap;
     this.isMap = isMap;
     this.jobId = jobId;
     this.jobId = jobId;
   }
   }
   
   
-  public JVMId (String jtIdentifier, int jobId, boolean isMap, int id) {
+  public JVMId (String jtIdentifier, int jobId, boolean isMap, long id) {
     this(new JobID(jtIdentifier, jobId), isMap, id);
     this(new JobID(jtIdentifier, jobId), isMap, id);
   }
   }
     
     
@@ -53,27 +55,50 @@ class JVMId extends ID {
   public JobID getJobId() {
   public JobID getJobId() {
     return jobId;
     return jobId;
   }
   }
+
+  @Override
   public boolean equals(Object o) {
   public boolean equals(Object o) {
-    if(o == null)
+    // Generated by IntelliJ IDEA 13.1.
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    JVMId jvmId1 = (JVMId) o;
+
+    if (isMap != jvmId1.isMap) {
       return false;
       return false;
-    if(o.getClass().equals(this.getClass())) {
-      JVMId that = (JVMId)o;
-      return this.id==that.id
-        && this.isMap == that.isMap
-        && this.jobId.equals(that.jobId);
     }
     }
-    else return false;
+    if (jvmId != jvmId1.jvmId) {
+      return false;
+    }
+    if (!jobId.equals(jvmId1.jobId)) {
+      return false;
+    }
+
+    return true;
   }
   }
 
 
-  /**Compare TaskInProgressIds by first jobIds, then by tip numbers. Reduces are 
-   * defined as greater then maps.*/
   @Override
   @Override
-  public int compareTo(org.apache.hadoop.mapreduce.ID o) {
-    JVMId that = (JVMId)o;
+  public int hashCode() {
+    // Generated by IntelliJ IDEA 13.1.
+    int result = (isMap ? 1 : 0);
+    result = 31 * result + jobId.hashCode();
+    result = 31 * result + (int) (jvmId ^ (jvmId >>> 32));
+    return result;
+  }
+
+  /**
+   * Compare TaskInProgressIds by first jobIds, then by tip numbers. Reduces are
+   * defined as greater then maps.
+   **/
+  public int compareTo(JVMId that) {
     int jobComp = this.jobId.compareTo(that.jobId);
     int jobComp = this.jobId.compareTo(that.jobId);
     if(jobComp == 0) {
     if(jobComp == 0) {
       if(this.isMap == that.isMap) {
       if(this.isMap == that.isMap) {
-        return this.id - that.id;
+        return Long.valueOf(this.jvmId).compareTo(that.jvmId);
       } else {
       } else {
         return this.isMap ? -1 : 1;
         return this.isMap ? -1 : 1;
       }
       }
@@ -87,6 +112,15 @@ class JVMId extends ID {
     return appendTo(new StringBuilder(JVM)).toString();
     return appendTo(new StringBuilder(JVM)).toString();
   }
   }
 
 
+  /**
+   * This method does NOT override org.apache.hadoop.mapred.ID to accept 64-bit
+   * ID to support work-preserving RM restart.
+   * @return 64-bit JVM id.
+   */
+  public long getId() {
+    return jvmId;
+  }
+
   /**
   /**
    * Add the unique id to the given StringBuilder.
    * Add the unique id to the given StringBuilder.
    * @param builder the builder to append to
    * @param builder the builder to append to
@@ -97,24 +131,17 @@ class JVMId extends ID {
                  append(SEPARATOR).
                  append(SEPARATOR).
                  append(isMap ? 'm' : 'r').
                  append(isMap ? 'm' : 'r').
                  append(SEPARATOR).
                  append(SEPARATOR).
-                 append(idFormat.format(id));
-  }
-  
-  @Override
-  public int hashCode() {
-    return jobId.hashCode() * 11 + id;
+                 append(idFormat.format(jvmId));
   }
   }
-  
-  @Override
+
   public void readFields(DataInput in) throws IOException {
   public void readFields(DataInput in) throws IOException {
-    super.readFields(in);
+    this.jvmId = in.readLong();
     this.jobId.readFields(in);
     this.jobId.readFields(in);
     this.isMap = in.readBoolean();
     this.isMap = in.readBoolean();
   }
   }
 
 
-  @Override
   public void write(DataOutput out) throws IOException {
   public void write(DataOutput out) throws IOException {
-    super.write(out);
+    out.writeLong(jvmId);
     jobId.write(out);
     jobId.write(out);
     out.writeBoolean(isMap);
     out.writeBoolean(isMap);
   }
   }

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

@@ -671,7 +671,8 @@ public class TestMRJobs {
           if (!foundAppMaster) {
           if (!foundAppMaster) {
             final ContainerId cid = ConverterUtils.toContainerId(
             final ContainerId cid = ConverterUtils.toContainerId(
                 containerPathComponent.getName());
                 containerPathComponent.getName());
-            foundAppMaster = (cid.getId() == 1);
+            foundAppMaster =
+                ((cid.getContainerId() & ContainerId.CONTAINER_ID_BITMASK)== 1);
           }
           }
 
 
           final FileStatus[] sysSiblings = localFs.globStatus(new Path(
           final FileStatus[] sysSiblings = localFs.globStatus(new Path(

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

@@ -348,6 +348,9 @@ Release 2.6.0 - UNRELEASED
     YARN-2583. Modified AggregatedLogDeletionService to be able to delete rolling
     YARN-2583. Modified AggregatedLogDeletionService to be able to delete rolling
     aggregated logs. (Xuan Gong via zjshen)
     aggregated logs. (Xuan Gong via zjshen)
 
 
+    YARN-2312. Deprecated old ContainerId#getId API and updated MapReduce to
+    use ContainerId#getContainerId instead. (Tsuyoshi OZAWA via jianhe)
+
   OPTIMIZATIONS
   OPTIMIZATIONS
 
 
   BUG FIXES
   BUG FIXES

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

@@ -35,6 +35,7 @@ import org.apache.hadoop.yarn.util.Records;
 @Public
 @Public
 @Stable
 @Stable
 public abstract class ContainerId implements Comparable<ContainerId>{
 public abstract class ContainerId implements Comparable<ContainerId>{
+  public static final long CONTAINER_ID_BITMASK = 0xffffffffffL;
   private static final Splitter _SPLITTER = Splitter.on('_').trimResults();
   private static final Splitter _SPLITTER = Splitter.on('_').trimResults();
   private static final String CONTAINER_PREFIX = "container";
   private static final String CONTAINER_PREFIX = "container";
   private static final String EPOCH_PREFIX = "e";
   private static final String EPOCH_PREFIX = "e";
@@ -81,6 +82,7 @@ public abstract class ContainerId implements Comparable<ContainerId>{
    * @return lower 32 bits of identifier of the <code>ContainerId</code>
    * @return lower 32 bits of identifier of the <code>ContainerId</code>
    */
    */
   @Public
   @Public
+  @Deprecated
   @Stable
   @Stable
   public abstract int getId();
   public abstract int getId();
 
 
@@ -184,7 +186,8 @@ public abstract class ContainerId implements Comparable<ContainerId>{
     sb.append(
     sb.append(
         appAttemptIdAndEpochFormat.get().format(
         appAttemptIdAndEpochFormat.get().format(
             getApplicationAttemptId().getAttemptId())).append("_");
             getApplicationAttemptId().getAttemptId())).append("_");
-    sb.append(containerIdFormat.get().format(0xffffffffffL & getContainerId()));
+    sb.append(containerIdFormat.get()
+        .format(CONTAINER_ID_BITMASK & getContainerId()));
     return sb.toString();
     return sb.toString();
   }
   }
 
 

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

@@ -219,6 +219,7 @@ public class TestNMClientAsync {
       actualStopFailureArray = new AtomicIntegerArray(expectedFailure);
       actualStopFailureArray = new AtomicIntegerArray(expectedFailure);
     }
     }
 
 
+    @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onContainerStarted(ContainerId containerId,
     public void onContainerStarted(ContainerId containerId,
         Map<String, ByteBuffer> allServiceResponse) {
         Map<String, ByteBuffer> allServiceResponse) {
@@ -242,6 +243,7 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
       throw new RuntimeException("Ignorable Exception");
     }
     }
 
 
+    @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onContainerStatusReceived(ContainerId containerId,
     public void onContainerStatusReceived(ContainerId containerId,
         ContainerStatus containerStatus) {
         ContainerStatus containerStatus) {
@@ -259,6 +261,7 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
       throw new RuntimeException("Ignorable Exception");
     }
     }
 
 
+    @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onContainerStopped(ContainerId containerId) {
     public void onContainerStopped(ContainerId containerId) {
       if (containerId.getId() >= expectedSuccess) {
       if (containerId.getId() >= expectedSuccess) {
@@ -273,6 +276,7 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
       throw new RuntimeException("Ignorable Exception");
     }
     }
 
 
+    @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onStartContainerError(ContainerId containerId, Throwable t) {
     public void onStartContainerError(ContainerId containerId, Throwable t) {
       // If the unexpected throwable comes from success callback functions, it
       // If the unexpected throwable comes from success callback functions, it
@@ -296,6 +300,7 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
       throw new RuntimeException("Ignorable Exception");
     }
     }
 
 
+    @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onStopContainerError(ContainerId containerId, Throwable t) {
     public void onStopContainerError(ContainerId containerId, Throwable t) {
       if (t instanceof RuntimeException) {
       if (t instanceof RuntimeException) {
@@ -316,6 +321,7 @@ public class TestNMClientAsync {
       throw new RuntimeException("Ignorable Exception");
       throw new RuntimeException("Ignorable Exception");
     }
     }
 
 
+    @SuppressWarnings("deprecation")
     @Override
     @Override
     public void onGetContainerStatusError(ContainerId containerId,
     public void onGetContainerStatusError(ContainerId containerId,
         Throwable t) {
         Throwable t) {

+ 1 - 0
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/apache/hadoop/yarn/api/records/impl/pb/ContainerIdPBImpl.java

@@ -47,6 +47,7 @@ public class ContainerIdPBImpl extends ContainerId {
     return proto;
     return proto;
   }
   }
 
 
+  @Deprecated
   @Override
   @Override
   public int getId() {
   public int getId() {
     Preconditions.checkNotNull(proto);
     Preconditions.checkNotNull(proto);

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

@@ -68,6 +68,7 @@ public class ApplicationHistoryStoreTestUtils {
       FinalApplicationStatus.UNDEFINED, YarnApplicationAttemptState.FINISHED));
       FinalApplicationStatus.UNDEFINED, YarnApplicationAttemptState.FINISHED));
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   protected void writeContainerStartData(ContainerId containerId)
   protected void writeContainerStartData(ContainerId containerId)
       throws IOException {
       throws IOException {
     store.containerStarted(ContainerStartData.newInstance(containerId,
     store.containerStarted(ContainerStartData.newInstance(containerId,

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

@@ -137,7 +137,8 @@ public class TestFileSystemApplicationHistoryStore extends
   private void testReadHistoryData(int num) throws IOException {
   private void testReadHistoryData(int num) throws IOException {
     testReadHistoryData(num, false, false);
     testReadHistoryData(num, false, false);
   }
   }
-  
+
+  @SuppressWarnings("deprecation")
   private void testReadHistoryData(
   private void testReadHistoryData(
       int num, boolean missingContainer, boolean missingApplicationAttempt)
       int num, boolean missingContainer, boolean missingApplicationAttempt)
           throws IOException {
           throws IOException {

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

@@ -130,6 +130,7 @@ public class TestMemoryApplicationHistoryStore extends
     }
     }
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   @Test
   @Test
   public void testReadWriteContainerHistory() throws Exception {
   public void testReadWriteContainerHistory() throws Exception {
     // Out of order
     // Out of order

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

@@ -265,6 +265,7 @@ public class TestAHSWebServices extends JerseyTest {
     assertEquals("incorrect number of elements", 5, array.length());
     assertEquals("incorrect number of elements", 5, array.length());
   }
   }
 
 
+  @SuppressWarnings("deprecation")
   @Test
   @Test
   public void testSingleContainer() throws Exception {
   public void testSingleContainer() throws Exception {
     ApplicationId appId = ApplicationId.newInstance(0, 1);
     ApplicationId appId = ApplicationId.newInstance(0, 1);

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

@@ -199,8 +199,8 @@ public class TestYarnServerApiClasses {
     original.setResponseId(1);
     original.setResponseId(1);
 
 
     NodeStatusPBImpl copy = new NodeStatusPBImpl(original.getProto());
     NodeStatusPBImpl copy = new NodeStatusPBImpl(original.getProto());
-    assertEquals(3, copy.getContainersStatuses().get(1).getContainerId()
-        .getId());
+    assertEquals(3L, copy.getContainersStatuses().get(1).getContainerId()
+        .getContainerId());
     assertEquals(3, copy.getKeepAliveApplications().get(0).getId());
     assertEquals(3, copy.getKeepAliveApplications().get(0).getId());
     assertEquals(1000, copy.getNodeHealthStatus().getLastHealthReportTime());
     assertEquals(1000, copy.getNodeHealthStatus().getLastHealthReportTime());
     assertEquals(9090, copy.getNodeId().getPort());
     assertEquals(9090, copy.getNodeId().getPort());

+ 4 - 2
hadoop-yarn-project/hadoop-yarn/hadoop-yarn-server/hadoop-yarn-server-nodemanager/src/main/java/org/apache/hadoop/yarn/server/nodemanager/containermanager/logaggregation/AppLogAggregatorImpl.java

@@ -430,7 +430,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     // AM Container only
     // AM Container only
     if (this.retentionPolicy
     if (this.retentionPolicy
         .equals(ContainerLogsRetentionPolicy.APPLICATION_MASTER_ONLY)) {
         .equals(ContainerLogsRetentionPolicy.APPLICATION_MASTER_ONLY)) {
-      if (containerId.getId() == 1) {
+      if ((containerId.getContainerId()
+          & ContainerId.CONTAINER_ID_BITMASK)== 1) {
         return true;
         return true;
       }
       }
       return false;
       return false;
@@ -439,7 +440,8 @@ public class AppLogAggregatorImpl implements AppLogAggregator {
     // AM + Failing containers
     // AM + Failing containers
     if (this.retentionPolicy
     if (this.retentionPolicy
         .equals(ContainerLogsRetentionPolicy.AM_AND_FAILED_CONTAINERS_ONLY)) {
         .equals(ContainerLogsRetentionPolicy.AM_AND_FAILED_CONTAINERS_ONLY)) {
-      if (containerId.getId() == 1) {
+      if ((containerId.getContainerId()
+          & ContainerId.CONTAINER_ID_BITMASK) == 1) {
         return true;
         return true;
       } else if(!wasContainerSuccessful) {
       } else if(!wasContainerSuccessful) {
         return true;
         return true;

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

@@ -665,13 +665,13 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertEquals(5, response.getSuccessfullyStartedContainers().size());
     Assert.assertEquals(5, response.getSuccessfullyStartedContainers().size());
     for (ContainerId id : response.getSuccessfullyStartedContainers()) {
     for (ContainerId id : response.getSuccessfullyStartedContainers()) {
       // Containers with odd id should succeed.
       // Containers with odd id should succeed.
-      Assert.assertEquals(1, id.getId() & 1);
+      Assert.assertEquals(1, id.getContainerId() & 1);
     }
     }
     Assert.assertEquals(5, response.getFailedRequests().size());
     Assert.assertEquals(5, response.getFailedRequests().size());
     for (Map.Entry<ContainerId, SerializedException> entry : response
     for (Map.Entry<ContainerId, SerializedException> entry : response
       .getFailedRequests().entrySet()) {
       .getFailedRequests().entrySet()) {
       // Containers with even id should fail.
       // Containers with even id should fail.
-      Assert.assertEquals(0, entry.getKey().getId() & 1);
+      Assert.assertEquals(0, entry.getKey().getContainerId() & 1);
       Assert.assertTrue(entry.getValue().getMessage()
       Assert.assertTrue(entry.getValue().getMessage()
         .contains(
         .contains(
           "Container " + entry.getKey() + " rejected as it is allocated by a previous RM"));
           "Container " + entry.getKey() + " rejected as it is allocated by a previous RM"));
@@ -718,13 +718,13 @@ public class TestContainerManager extends BaseContainerManagerTest {
     Assert.assertEquals(5, statusResponse.getContainerStatuses().size());
     Assert.assertEquals(5, statusResponse.getContainerStatuses().size());
     for (ContainerStatus status : statusResponse.getContainerStatuses()) {
     for (ContainerStatus status : statusResponse.getContainerStatuses()) {
       // Containers with odd id should succeed
       // Containers with odd id should succeed
-      Assert.assertEquals(1, status.getContainerId().getId() & 1);
+      Assert.assertEquals(1, status.getContainerId().getContainerId() & 1);
     }
     }
     Assert.assertEquals(5, statusResponse.getFailedRequests().size());
     Assert.assertEquals(5, statusResponse.getFailedRequests().size());
     for (Map.Entry<ContainerId, SerializedException> entry : statusResponse
     for (Map.Entry<ContainerId, SerializedException> entry : statusResponse
       .getFailedRequests().entrySet()) {
       .getFailedRequests().entrySet()) {
       // Containers with even id should fail.
       // Containers with even id should fail.
-      Assert.assertEquals(0, entry.getKey().getId() & 1);
+      Assert.assertEquals(0, entry.getKey().getContainerId() & 1);
       Assert.assertTrue(entry.getValue().getMessage()
       Assert.assertTrue(entry.getValue().getMessage()
         .contains("Reject this container"));
         .contains("Reject this container"));
     }
     }
@@ -738,13 +738,13 @@ public class TestContainerManager extends BaseContainerManagerTest {
       .size());
       .size());
     for (ContainerId id : stopResponse.getSuccessfullyStoppedContainers()) {
     for (ContainerId id : stopResponse.getSuccessfullyStoppedContainers()) {
       // Containers with odd id should succeed.
       // Containers with odd id should succeed.
-      Assert.assertEquals(1, id.getId() & 1);
+      Assert.assertEquals(1, id.getContainerId() & 1);
     }
     }
     Assert.assertEquals(5, stopResponse.getFailedRequests().size());
     Assert.assertEquals(5, stopResponse.getFailedRequests().size());
     for (Map.Entry<ContainerId, SerializedException> entry : stopResponse
     for (Map.Entry<ContainerId, SerializedException> entry : stopResponse
       .getFailedRequests().entrySet()) {
       .getFailedRequests().entrySet()) {
       // Containers with even id should fail.
       // Containers with even id should fail.
-      Assert.assertEquals(0, entry.getKey().getId() & 1);
+      Assert.assertEquals(0, entry.getKey().getContainerId() & 1);
       Assert.assertTrue(entry.getValue().getMessage()
       Assert.assertTrue(entry.getValue().getMessage()
         .contains("Reject this container"));
         .contains("Reject this container"));
     }
     }

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

@@ -49,7 +49,7 @@ import org.mockito.ArgumentMatcher;
 
 
 public class TestLocalizedResource {
 public class TestLocalizedResource {
 
 
-  static ContainerId getMockContainer(int id) {
+  static ContainerId getMockContainer(long id) {
     ApplicationId appId = mock(ApplicationId.class);
     ApplicationId appId = mock(ApplicationId.class);
     when(appId.getClusterTimestamp()).thenReturn(314159265L);
     when(appId.getClusterTimestamp()).thenReturn(314159265L);
     when(appId.getId()).thenReturn(3);
     when(appId.getId()).thenReturn(3);
@@ -57,7 +57,7 @@ public class TestLocalizedResource {
     when(appAttemptId.getApplicationId()).thenReturn(appId);
     when(appAttemptId.getApplicationId()).thenReturn(appId);
     when(appAttemptId.getAttemptId()).thenReturn(0);
     when(appAttemptId.getAttemptId()).thenReturn(0);
     ContainerId container = mock(ContainerId.class);
     ContainerId container = mock(ContainerId.class);
-    when(container.getId()).thenReturn(id);
+    when(container.getContainerId()).thenReturn(id);
     when(container.getApplicationAttemptId()).thenReturn(appAttemptId);
     when(container.getApplicationAttemptId()).thenReturn(appAttemptId);
     return container;
     return container;
   }
   }
@@ -77,7 +77,7 @@ public class TestLocalizedResource {
       // mock resource
       // mock resource
       LocalResource apiRsrc = createMockResource();
       LocalResource apiRsrc = createMockResource();
 
 
-      final ContainerId container0 = getMockContainer(0);
+      final ContainerId container0 = getMockContainer(0L);
       final Credentials creds0 = new Credentials();
       final Credentials creds0 = new Credentials();
       final LocalResourceVisibility vis0 = LocalResourceVisibility.PRIVATE;
       final LocalResourceVisibility vis0 = LocalResourceVisibility.PRIVATE;
       final LocalizerContext ctxt0 =
       final LocalizerContext ctxt0 =
@@ -96,7 +96,7 @@ public class TestLocalizedResource {
 
 
       // Register C1, verify request event
       // Register C1, verify request event
       final Credentials creds1 = new Credentials();
       final Credentials creds1 = new Credentials();
-      final ContainerId container1 = getMockContainer(1);
+      final ContainerId container1 = getMockContainer(1L);
       final LocalizerContext ctxt1 =
       final LocalizerContext ctxt1 =
         new LocalizerContext("yak", container1, creds1);
         new LocalizerContext("yak", container1, creds1);
       final LocalResourceVisibility vis1 = LocalResourceVisibility.PUBLIC;
       final LocalResourceVisibility vis1 = LocalResourceVisibility.PUBLIC;
@@ -120,13 +120,13 @@ public class TestLocalizedResource {
       assertEquals(ResourceState.DOWNLOADING, local.getState());
       assertEquals(ResourceState.DOWNLOADING, local.getState());
 
 
       // Register C2, C3
       // Register C2, C3
-      final ContainerId container2 = getMockContainer(2);
+      final ContainerId container2 = getMockContainer(2L);
       final LocalResourceVisibility vis2 = LocalResourceVisibility.PRIVATE;
       final LocalResourceVisibility vis2 = LocalResourceVisibility.PRIVATE;
       final Credentials creds2 = new Credentials();
       final Credentials creds2 = new Credentials();
       final LocalizerContext ctxt2 =
       final LocalizerContext ctxt2 =
         new LocalizerContext("yak", container2, creds2);
         new LocalizerContext("yak", container2, creds2);
 
 
-      final ContainerId container3 = getMockContainer(3);
+      final ContainerId container3 = getMockContainer(3L);
       final LocalResourceVisibility vis3 = LocalResourceVisibility.PRIVATE;
       final LocalResourceVisibility vis3 = LocalResourceVisibility.PRIVATE;
       final Credentials creds3 = new Credentials();
       final Credentials creds3 = new Credentials();
       final LocalizerContext ctxt3 =
       final LocalizerContext ctxt3 =
@@ -159,7 +159,7 @@ public class TestLocalizedResource {
       assertEquals(ResourceState.LOCALIZED, local.getState());
       assertEquals(ResourceState.LOCALIZED, local.getState());
 
 
       // Register C4, verify notification
       // Register C4, verify notification
-      final ContainerId container4 = getMockContainer(4);
+      final ContainerId container4 = getMockContainer(4L);
       final Credentials creds4 = new Credentials();
       final Credentials creds4 = new Credentials();
       final LocalizerContext ctxt4 =
       final LocalizerContext ctxt4 =
         new LocalizerContext("yak", container4, creds4);
         new LocalizerContext("yak", container4, creds4);

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

@@ -134,7 +134,7 @@ public class MockNM {
   }
   }
 
 
   public NodeHeartbeatResponse nodeHeartbeat(ApplicationAttemptId attemptId,
   public NodeHeartbeatResponse nodeHeartbeat(ApplicationAttemptId attemptId,
-      int containerId, ContainerState containerState) throws Exception {
+      long containerId, ContainerState containerState) throws Exception {
     HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate =
     HashMap<ApplicationId, List<ContainerStatus>> nodeUpdate =
         new HashMap<ApplicationId, List<ContainerStatus>>(1);
         new HashMap<ApplicationId, List<ContainerStatus>>(1);
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(
     ContainerStatus containerStatus = BuilderUtils.newContainerStatus(

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

@@ -169,7 +169,7 @@ public class TestContainerResourceUsage {
     // launch the 2nd and 3rd containers.
     // launch the 2nd and 3rd containers.
     for (Container c : conts) {
     for (Container c : conts) {
       nm.nodeHeartbeat(attempt0.getAppAttemptId(),
       nm.nodeHeartbeat(attempt0.getAppAttemptId(),
-                       c.getId().getId(), ContainerState.RUNNING);
+                       c.getId().getContainerId(), ContainerState.RUNNING);
       rm0.waitForState(nm, c.getId(), RMContainerState.RUNNING);
       rm0.waitForState(nm, c.getId(), RMContainerState.RUNNING);
     }
     }
 
 
@@ -185,9 +185,9 @@ public class TestContainerResourceUsage {
 
 
     // Stop all non-AM containers
     // Stop all non-AM containers
     for (Container c : conts) {
     for (Container c : conts) {
-      if (c.getId().getId() == 1) continue;
+      if (c.getId().getContainerId() == 1) continue;
       nm.nodeHeartbeat(attempt0.getAppAttemptId(),
       nm.nodeHeartbeat(attempt0.getAppAttemptId(),
-                       c.getId().getId(), ContainerState.COMPLETE);
+                       c.getId().getContainerId(), ContainerState.COMPLETE);
       rm0.waitForState(nm, c.getId(), RMContainerState.COMPLETED);
       rm0.waitForState(nm, c.getId(), RMContainerState.COMPLETED);
     }
     }
 
 
@@ -198,7 +198,7 @@ public class TestContainerResourceUsage {
     // that all containers will complete prior to saving.
     // that all containers will complete prior to saving.
     ContainerId cId = ContainerId.newInstance(attempt0.getAppAttemptId(), 1);
     ContainerId cId = ContainerId.newInstance(attempt0.getAppAttemptId(), 1);
     nm.nodeHeartbeat(attempt0.getAppAttemptId(),
     nm.nodeHeartbeat(attempt0.getAppAttemptId(),
-                 cId.getId(), ContainerState.COMPLETE);
+                 cId.getContainerId(), ContainerState.COMPLETE);
     rm0.waitForState(nm, cId, RMContainerState.COMPLETED);
     rm0.waitForState(nm, cId, RMContainerState.COMPLETED);
 
 
     // Check that the container metrics match those from the app usage report.
     // Check that the container metrics match those from the app usage report.
@@ -280,7 +280,7 @@ public class TestContainerResourceUsage {
     ContainerId containerId2 =
     ContainerId containerId2 =
         ContainerId.newInstance(am0.getApplicationAttemptId(), 2);
         ContainerId.newInstance(am0.getApplicationAttemptId(), 2);
     nm.nodeHeartbeat(am0.getApplicationAttemptId(),
     nm.nodeHeartbeat(am0.getApplicationAttemptId(),
-                      containerId2.getId(), ContainerState.RUNNING);
+                      containerId2.getContainerId(), ContainerState.RUNNING);
     rm.waitForState(nm, containerId2, RMContainerState.RUNNING);
     rm.waitForState(nm, containerId2, RMContainerState.RUNNING);
 
 
     // Capture the containers here so the metrics can be calculated after the
     // Capture the containers here so the metrics can be calculated after the
@@ -295,7 +295,7 @@ public class TestContainerResourceUsage {
     ContainerId amContainerId =
     ContainerId amContainerId =
         app.getCurrentAppAttempt().getMasterContainer().getId();
         app.getCurrentAppAttempt().getMasterContainer().getId();
     nm.nodeHeartbeat(am0.getApplicationAttemptId(),
     nm.nodeHeartbeat(am0.getApplicationAttemptId(),
-                      amContainerId.getId(), ContainerState.COMPLETE);
+                      amContainerId.getContainerId(), ContainerState.COMPLETE);
     am0.waitForState(RMAppAttemptState.FAILED);
     am0.waitForState(RMAppAttemptState.FAILED);
 
 
     long memorySeconds = 0;
     long memorySeconds = 0;
@@ -365,7 +365,7 @@ public class TestContainerResourceUsage {
     // earlier attempt's attemptId
     // earlier attempt's attemptId
     amContainerId = app.getCurrentAppAttempt().getMasterContainer().getId();
     amContainerId = app.getCurrentAppAttempt().getMasterContainer().getId();
     nm.nodeHeartbeat(am0.getApplicationAttemptId(),
     nm.nodeHeartbeat(am0.getApplicationAttemptId(),
-                      amContainerId.getId(), ContainerState.COMPLETE);
+                      amContainerId.getContainerId(), ContainerState.COMPLETE);
     
     
     MockRM.finishAMAndVerifyAppState(app, rm, nm, am1);
     MockRM.finishAMAndVerifyAppState(app, rm, nm, am1);
 
 

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

@@ -214,7 +214,8 @@ public class TestRM extends ParameterizedSchedulerTestBase {
     nm1.nodeHeartbeat(true);
     nm1.nodeHeartbeat(true);
     MockAM am = MockRM.launchAM(app, rm, nm1);
     MockAM am = MockRM.launchAM(app, rm, nm1);
     // am container Id not equal to 1.
     // am container Id not equal to 1.
-    Assert.assertTrue(attempt.getMasterContainer().getId().getId() != 1);
+    Assert.assertTrue(
+        attempt.getMasterContainer().getId().getContainerId() != 1);
     // NMSecretManager doesn't record the node on which the am is allocated.
     // NMSecretManager doesn't record the node on which the am is allocated.
     Assert.assertFalse(rm.getRMContext().getNMTokenSecretManager()
     Assert.assertFalse(rm.getRMContext().getNMTokenSecretManager()
       .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(),
       .isApplicationAttemptNMTokenPresent(attempt.getAppAttemptId(),
@@ -382,11 +383,13 @@ public class TestRM extends ParameterizedSchedulerTestBase {
       am.unregisterAppAttempt();
       am.unregisterAppAttempt();
       // marking all the containers as finished.
       // marking all the containers as finished.
       for (Container container : containersReceivedForNM1) {
       for (Container container : containersReceivedForNM1) {
-        nm1.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(),
+        nm1.nodeHeartbeat(attempt.getAppAttemptId(),
+            container.getId().getContainerId(),
             ContainerState.COMPLETE);
             ContainerState.COMPLETE);
       }
       }
       for (Container container : containersReceivedForNM2) {
       for (Container container : containersReceivedForNM2) {
-        nm2.nodeHeartbeat(attempt.getAppAttemptId(), container.getId().getId(),
+        nm2.nodeHeartbeat(attempt.getAppAttemptId(),
+            container.getId().getContainerId(),
             ContainerState.COMPLETE);
             ContainerState.COMPLETE);
       }
       }
       nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1,
       nm1.nodeHeartbeat(am.getApplicationAttemptId(), 1,

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

@@ -167,7 +167,8 @@ public class TestUtils {
     LOG.info("node = " + host + " avail=" + node.getAvailableResource());
     LOG.info("node = " + host + " avail=" + node.getAvailableResource());
     return node;
     return node;
   }
   }
-  
+
+  @SuppressWarnings("deprecation")
   public static ContainerId getMockContainerId(FiCaSchedulerApp application) {
   public static ContainerId getMockContainerId(FiCaSchedulerApp application) {
     ContainerId containerId = mock(ContainerId.class);
     ContainerId containerId = mock(ContainerId.class);
     doReturn(application.getApplicationAttemptId()).
     doReturn(application.getApplicationAttemptId()).