You are viewing a plain text version of this content. The canonical link for it is here.
Posted to mapreduce-commits@hadoop.apache.org by vi...@apache.org on 2013/05/30 22:19:54 UTC

svn commit: r1487995 - in /hadoop/common/branches/branch-2/hadoop-mapreduce-project: ./ hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/ hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/j...

Author: vinodkv
Date: Thu May 30 20:19:53 2013
New Revision: 1487995

URL: http://svn.apache.org/r1487995
Log:
YARN-716. Making ApplicationID immutable. Contributed by Siddharth Seth.
MAPREDUCE-5282. Updating MR App to use immutable ApplicationID after YARN-716. Contributed by Siddharth Seth.
svn merge --ignore-ancestry -c 1487994 ../../trunk/

Modified:
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/CHANGES.txt
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestShufflePlugin.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
    hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/CHANGES.txt
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/CHANGES.txt?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/CHANGES.txt (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/CHANGES.txt Thu May 30 20:19:53 2013
@@ -306,6 +306,9 @@ Release 2.0.5-beta - UNRELEASED
     MAPREDUCE-5261. Fix issues in TestRMContainerAllocator after YARN-617.
     (Omkar Vinit Joshi via vinodkv)
 
+    MAPREDUCE-5282. Updating MR App to use immutable ApplicationID after
+    YARN-716. (Siddharth Seth via vinodkv)
+
   BREAKDOWN OF HADOOP-8562 SUBTASKS
 
     MAPREDUCE-4739. Some MapReduce tests fail to find winutils.

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/MRApp.java Thu May 30 20:19:53 2013
@@ -128,9 +128,7 @@ public class MRApp extends MRAppMaster {
   static ApplicationId applicationId;
 
   static {
-    applicationId = recordFactory.newRecordInstance(ApplicationId.class);
-    applicationId.setClusterTimestamp(0);
-    applicationId.setId(0);
+    applicationId = ApplicationId.newInstance(0, 0);
   }
 
   public MRApp(int maps, int reduces, boolean autoComplete, String testName,

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestRuntimeEstimators.java Thu May 30 20:19:53 2013
@@ -789,9 +789,7 @@ public class TestRuntimeEstimators {
     private final Map<JobId, Job> allJobs;
 
     MyAppContext(int numberMaps, int numberReduces) {
-      myApplicationID = recordFactory.newRecordInstance(ApplicationId.class);
-      myApplicationID.setClusterTimestamp(clock.getTime());
-      myApplicationID.setId(1);
+      myApplicationID = ApplicationId.newInstance(clock.getTime(), 1);
 
       myAppAttemptID = recordFactory
           .newRecordInstance(ApplicationAttemptId.class);

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/TestStagingCleanup.java Thu May 30 20:19:53 2013
@@ -84,9 +84,8 @@ import org.junit.Test;
      ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
          ApplicationAttemptId.class);
      attemptId.setAttemptId(0);
-     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
-     appId.setClusterTimestamp(System.currentTimeMillis());
-     appId.setId(0);
+     ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
+        0);
      attemptId.setApplicationId(appId);
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
@@ -113,9 +112,8 @@ import org.junit.Test;
      ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
          ApplicationAttemptId.class);
      attemptId.setAttemptId(0);
-     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
-     appId.setClusterTimestamp(System.currentTimeMillis());
-     appId.setId(0);
+     ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
+         0);
      attemptId.setApplicationId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
      Assert.assertTrue(MRJobConfig.DEFAULT_MR_AM_MAX_ATTEMPTS > 1);
@@ -141,9 +139,8 @@ import org.junit.Test;
      ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
          ApplicationAttemptId.class);
      attemptId.setAttemptId(1);
-     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
-     appId.setClusterTimestamp(System.currentTimeMillis());
-     appId.setId(0);
+     ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
+         0);
      attemptId.setApplicationId(appId);
      ContainerAllocator mockAlloc = mock(ContainerAllocator.class);
      MRAppMaster appMaster = new TestMRApp(attemptId, mockAlloc,
@@ -169,9 +166,8 @@ import org.junit.Test;
      ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
          ApplicationAttemptId.class);
      attemptId.setAttemptId(0);
-     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
-     appId.setClusterTimestamp(System.currentTimeMillis());
-     appId.setId(0);
+     ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
+         0);
      attemptId.setApplicationId(appId);
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);
@@ -197,9 +193,8 @@ import org.junit.Test;
      ApplicationAttemptId attemptId = recordFactory.newRecordInstance(
          ApplicationAttemptId.class);
      attemptId.setAttemptId(1);
-     ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
-     appId.setClusterTimestamp(System.currentTimeMillis());
-     appId.setId(0);
+     ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
+         0);
      attemptId.setApplicationId(appId);
      JobId jobid = recordFactory.newRecordInstance(JobId.class);
      jobid.setAppId(appId);

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/job/impl/TestTaskImpl.java Thu May 30 20:19:53 2013
@@ -224,9 +224,7 @@ public class TestTaskImpl {
     metrics = mock(MRAppMetrics.class);  
     dataLocations = new String[1];
     
-    appId = Records.newRecord(ApplicationId.class);
-    appId.setClusterTimestamp(System.currentTimeMillis());
-    appId.setId(1);
+    appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
 
     jobId = Records.newRecord(JobId.class);
     jobId.setId(1);

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/test/java/org/apache/hadoop/mapreduce/v2/app/webapp/TestAppController.java Thu May 30 20:19:53 2013
@@ -27,7 +27,6 @@ import org.apache.commons.lang.StringUti
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.mapreduce.v2.app.AppContext;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.yarn.webapp.Controller.RequestContext;
 import org.junit.Before;
 import org.junit.Test;
@@ -41,7 +40,7 @@ public class TestAppController {
   public void setUp() {
     AppContext context = mock(AppContext.class);
     when(context.getApplicationID()).thenReturn(
-        Records.newRecord(ApplicationId.class));
+        ApplicationId.newInstance(0, 0));
     App app = new App(context);
     Configuration conf = new Configuration();
     ctx = mock(RequestContext.class);

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/TypeConverter.java Thu May 30 20:19:53 2013
@@ -76,9 +76,8 @@ public class TypeConverter {
     JobId jobId = recordFactory.newRecordInstance(JobId.class);
     jobId.setId(id.getId()); //currently there is 1-1 mapping between appid and jobid
 
-    ApplicationId appId = recordFactory.newRecordInstance(ApplicationId.class);
-    appId.setId(id.getId());
-    appId.setClusterTimestamp(toClusterTimeStamp(id.getJtIdentifier()));
+    ApplicationId appId = ApplicationId.newInstance(
+        toClusterTimeStamp(id.getJtIdentifier()), id.getId());
     jobId.setAppId(appId);
     return jobId;
   }

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/TestTypeConverter.java Thu May 30 20:19:53 2013
@@ -17,6 +17,9 @@
  */
 package org.apache.hadoop.mapreduce;
 
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
 import java.util.ArrayList;
 import java.util.List;
 
@@ -28,18 +31,13 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
-import org.apache.hadoop.yarn.api.records.YarnApplicationState;
 import org.apache.hadoop.yarn.api.records.ApplicationReport;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationIdPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationReportPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ApplicationResourceUsageReportPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.QueueInfoPBImpl;
-import org.apache.hadoop.yarn.api.records.impl.pb.ResourcePBImpl;
+import org.apache.hadoop.yarn.api.records.ApplicationResourceUsageReport;
+import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.api.records.QueueState;
-
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.api.records.YarnApplicationState;
+import org.apache.hadoop.yarn.util.Records;
 import org.junit.Test;
 import org.mockito.Mockito;
 
@@ -74,14 +72,16 @@ public class TestTypeConverter {
   public void testFromYarn() throws Exception {
     int appStartTime = 612354;
     YarnApplicationState state = YarnApplicationState.RUNNING;
-    ApplicationId applicationId = new ApplicationIdPBImpl();
-    ApplicationReportPBImpl applicationReport = new ApplicationReportPBImpl();
+    ApplicationId applicationId = ApplicationId.newInstance(0, 0);
+    ApplicationReport applicationReport = Records
+        .newRecord(ApplicationReport.class);
     applicationReport.setApplicationId(applicationId);
     applicationReport.setYarnApplicationState(state);
     applicationReport.setStartTime(appStartTime);
     applicationReport.setUser("TestTypeConverter-user");
-    ApplicationResourceUsageReportPBImpl appUsageRpt = new ApplicationResourceUsageReportPBImpl();
-    ResourcePBImpl r = new ResourcePBImpl();
+    ApplicationResourceUsageReport appUsageRpt = Records
+        .newRecord(ApplicationResourceUsageReport.class);
+    Resource r = Records.newRecord(Resource.class);
     r.setMemory(2048);
     appUsageRpt.setNeededResources(r);
     appUsageRpt.setNumReservedContainers(1);
@@ -107,8 +107,9 @@ public class TestTypeConverter {
     when(mockReport.getUser()).thenReturn("dummy-user");
     when(mockReport.getQueue()).thenReturn("dummy-queue");
     String jobFile = "dummy-path/job.xml";
-    ApplicationResourceUsageReportPBImpl appUsageRpt = new ApplicationResourceUsageReportPBImpl();
-    ResourcePBImpl r = new ResourcePBImpl();
+    ApplicationResourceUsageReport appUsageRpt = Records
+        .newRecord(ApplicationResourceUsageReport.class);
+    Resource r = Records.newRecord(Resource.class);
     r.setMemory(2048);
     appUsageRpt.setNeededResources(r);
     appUsageRpt.setNumReservedContainers(1);
@@ -134,7 +135,8 @@ public class TestTypeConverter {
 
   @Test
   public void testFromYarnQueueInfo() {
-    org.apache.hadoop.yarn.api.records.QueueInfo queueInfo = new QueueInfoPBImpl();
+    org.apache.hadoop.yarn.api.records.QueueInfo queueInfo = Records
+        .newRecord(org.apache.hadoop.yarn.api.records.QueueInfo.class);
     queueInfo.setQueueState(org.apache.hadoop.yarn.api.records.QueueState.STOPPED);
     org.apache.hadoop.mapreduce.QueueInfo returned =
       TypeConverter.fromYarn(queueInfo, new Configuration());

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/test/java/org/apache/hadoop/mapreduce/v2/util/TestMRApps.java Thu May 30 20:19:53 2013
@@ -82,7 +82,7 @@ public class TestMRApps {
   @Test (timeout = 120000)
   public void testJobIDtoString() {
     JobId jid = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobId.class);
-    jid.setAppId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class));
+    jid.setAppId(ApplicationId.newInstance(0, 0));
     assertEquals("job_0_0000", MRApps.toString(jid));
   }
 
@@ -104,7 +104,7 @@ public class TestMRApps {
   public void testTaskIDtoString() {
     TaskId tid = RecordFactoryProvider.getRecordFactory(null).newRecordInstance(TaskId.class);
     tid.setJobId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobId.class));
-    tid.getJobId().setAppId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class));
+    tid.getJobId().setAppId(ApplicationId.newInstance(0, 0));
     tid.setTaskType(TaskType.MAP);
     TaskType type = tid.getTaskType();
     System.err.println(type);
@@ -146,7 +146,7 @@ public class TestMRApps {
     taid.setTaskId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(TaskId.class));
     taid.getTaskId().setTaskType(TaskType.MAP);
     taid.getTaskId().setJobId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(JobId.class));
-    taid.getTaskId().getJobId().setAppId(RecordFactoryProvider.getRecordFactory(null).newRecordInstance(ApplicationId.class));
+    taid.getTaskId().getJobId().setAppId(ApplicationId.newInstance(0, 0));
     assertEquals("attempt_0_0000_m_000000_0", MRApps.toString(taid));
   }
 

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestShufflePlugin.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestShufflePlugin.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestShufflePlugin.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/test/java/org/apache/hadoop/mapreduce/TestShufflePlugin.java Thu May 30 20:19:53 2013
@@ -21,7 +21,6 @@ package org.apache.hadoop.mapreduce;
 import org.junit.Test;
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.mapred.Task.CombineOutputCollector;
@@ -30,7 +29,6 @@ import org.apache.hadoop.util.Progress;
 import org.apache.hadoop.mapred.Reporter;
 import org.apache.hadoop.util.ReflectionUtils;
 import org.apache.hadoop.mapreduce.task.reduce.Shuffle;
-import org.apache.hadoop.mapred.Counters;
 import org.apache.hadoop.mapred.Counters.Counter;
 import org.apache.hadoop.mapred.MapOutputFile;
 import org.apache.hadoop.mapred.JobConf;
@@ -40,7 +38,6 @@ import org.apache.hadoop.mapred.TaskStat
 import org.apache.hadoop.mapred.TaskUmbilicalProtocol;
 import org.apache.hadoop.mapred.ShuffleConsumerPlugin;
 import org.apache.hadoop.mapred.RawKeyValueIterator;
-import org.apache.hadoop.mapred.Reducer;
 
 /**
   * A JUnit for testing availability and accessibility of shuffle related API.
@@ -181,10 +178,6 @@ public class TestShufflePlugin<K, V> {
    * AuxiliaryService(s) which are "Shuffle-Providers" (ShuffleHandler and 3rd party plugins)
    */
   public void testProviderApi() {
-
-    ApplicationId mockApplicationId = mock(ApplicationId.class);
-    mockApplicationId.setClusterTimestamp(new Long(10));
-    mockApplicationId.setId(mock(JobID.class).getId());
     LocalDirAllocator mockLocalDirAllocator = mock(LocalDirAllocator.class);
     JobConf mockJobConf = mock(JobConf.class);
     try {

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-hs/src/main/java/org/apache/hadoop/mapreduce/v2/hs/JobHistory.java Thu May 30 20:19:53 2013
@@ -74,8 +74,7 @@ public class JobHistory extends Abstract
   public void init(Configuration conf) throws YarnException {
     LOG.info("JobHistory Init");
     this.conf = conf;
-    this.appID = RecordFactoryProvider.getRecordFactory(conf)
-        .newRecordInstance(ApplicationId.class);
+    this.appID = ApplicationId.newInstance(0, 0);
     this.appAttemptID = RecordFactoryProvider.getRecordFactory(conf)
         .newRecordInstance(ApplicationAttemptId.class);
 

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestResourceMgrDelegate.java Thu May 30 20:19:53 2013
@@ -129,7 +129,7 @@ public class TestResourceMgrDelegate {
     ApplicationResourceUsageReport appResources = Mockito
         .mock(ApplicationResourceUsageReport.class);
     Mockito.when(appReport.getApplicationId()).thenReturn(
-        Records.newRecord(ApplicationId.class));
+        ApplicationId.newInstance(0, 0));
     Mockito.when(appResources.getNeededResources()).thenReturn(
         Records.newRecord(Resource.class));
     Mockito.when(appResources.getReservedResources()).thenReturn(

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-jobclient/src/test/java/org/apache/hadoop/mapred/TestYARNRunner.java Thu May 30 20:19:53 2013
@@ -140,9 +140,7 @@ public class TestYARNRunner extends Test
         ).when(yarnRunner).createApplicationSubmissionContext(any(Configuration.class),
             any(String.class), any(Credentials.class));
 
-    appId = recordFactory.newRecordInstance(ApplicationId.class);
-    appId.setClusterTimestamp(System.currentTimeMillis());
-    appId.setId(1);
+    appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
     jobId = TypeConverter.fromYarn(appId);
     if (testWorkDir.exists()) {
       FileContext.getLocalFSFileContext().delete(new Path(testWorkDir.toString()), true);

Modified: hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java?rev=1487995&r1=1487994&r2=1487995&view=diff
==============================================================================
--- hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java (original)
+++ hadoop/common/branches/branch-2/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-shuffle/src/main/java/org/apache/hadoop/mapred/ShuffleHandler.java Thu May 30 20:19:53 2013
@@ -79,7 +79,6 @@ import org.apache.hadoop.yarn.server.nod
 import org.apache.hadoop.yarn.server.nodemanager.containermanager.localizer.ContainerLocalizer;
 import org.apache.hadoop.yarn.service.AbstractService;
 import org.apache.hadoop.yarn.util.ConverterUtils;
-import org.apache.hadoop.yarn.util.Records;
 import org.jboss.netty.bootstrap.ServerBootstrap;
 import org.jboss.netty.buffer.ChannelBuffers;
 import org.jboss.netty.channel.Channel;
@@ -549,9 +548,8 @@ public class ShuffleHandler extends Abst
       // $x/$user/appcache/$appId/output/$mapId
       // TODO: Once Shuffle is out of NM, this can use MR APIs to convert between App and Job
       JobID jobID = JobID.forName(jobId);
-      ApplicationId appID = Records.newRecord(ApplicationId.class);
-      appID.setClusterTimestamp(Long.parseLong(jobID.getJtIdentifier()));
-      appID.setId(jobID.getId());
+      ApplicationId appID = ApplicationId.newInstance(
+          Long.parseLong(jobID.getJtIdentifier()), jobID.getId());
       final String base =
           ContainerLocalizer.USERCACHE + "/" + user + "/"
               + ContainerLocalizer.APPCACHE + "/"