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 ss...@apache.org on 2012/10/19 21:01:11 UTC

svn commit: r1400232 [2/2] - in /hadoop/common/branches/MR-3902/hadoop-mapreduce-project: ./ hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/ hadoop-mapreduce-client/hadoop-mapreduce-client-app2/sr...

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/job/impl/TestTaskAttempt.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/job/impl/TestTaskAttempt.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/job/impl/TestTaskAttempt.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/job/impl/TestTaskAttempt.java Fri Oct 19 19:01:10 2012
@@ -20,6 +20,7 @@ package org.apache.hadoop.mapreduce.v2.a
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
+import static org.mockito.Mockito.doReturn;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.spy;
 import static org.mockito.Mockito.times;
@@ -28,7 +29,6 @@ import static org.mockito.Mockito.when;
 
 import java.io.IOException;
 import java.net.InetSocketAddress;
-import java.util.Arrays;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
@@ -36,23 +36,18 @@ import java.util.Map;
 import junit.framework.Assert;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.RawLocalFileSystem;
-import org.apache.hadoop.io.DataInputByteBuffer;
-import org.apache.hadoop.io.Text;
 import org.apache.hadoop.mapred.JobConf;
-import org.apache.hadoop.mapred.MapTaskAttemptImpl;
-import org.apache.hadoop.mapred.WrappedJvmID;
+import org.apache.hadoop.mapred.MapTaskAttemptImpl2;
 import org.apache.hadoop.mapreduce.JobCounter;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.TypeConverter;
+import org.apache.hadoop.mapreduce.jobhistory.ContainerHeartbeatHandler;
 import org.apache.hadoop.mapreduce.jobhistory.JobHistoryEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskAttemptUnsuccessfulCompletion;
-import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
 import org.apache.hadoop.mapreduce.split.JobSplit.TaskSplitMetaInfo;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
@@ -66,33 +61,34 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.ControlledClock;
 import org.apache.hadoop.mapreduce.v2.app2.MRApp;
 import org.apache.hadoop.mapreduce.v2.app2.TaskAttemptListener;
+import org.apache.hadoop.mapreduce.v2.app2.TaskHeartbeatHandler;
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
 import org.apache.hadoop.mapreduce.v2.app2.job.Task;
 import org.apache.hadoop.mapreduce.v2.app2.job.TaskAttempt;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.JobEvent;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.JobEventType;
-import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptContainerAssignedEvent;
-import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptContainerLaunchedEvent;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptDiagnosticsUpdateEvent;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEvent;
+import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventContainerTerminated;
+import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventContainerTerminating;
+import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventFailRequest;
+import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventKillRequest;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventType;
-import org.apache.hadoop.mapreduce.v2.app2.rm.ContainerRequestEvent;
+import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptRemoteStartEvent;
+import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptScheduleEvent;
+import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTALaunchRequestEvent;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerMap;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
-import org.apache.hadoop.security.token.TokenIdentifier;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
 import org.apache.hadoop.yarn.SystemClock;
-import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
 import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Resource;
 import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.BuilderUtils;
@@ -101,82 +97,6 @@ import org.mockito.ArgumentCaptor;
 
 @SuppressWarnings({"unchecked", "rawtypes"})
 public class TestTaskAttempt{
-  @Test
-  public void testAttemptContainerRequest() throws Exception {
-    //WARNING: This test must run first.  This is because there is an 
-    // optimization where the credentials passed in are cached statically so 
-    // they do not need to be recomputed when creating a new 
-    // ContainerLaunchContext. if other tests run first this code will cache
-    // their credentials and this test will fail trying to look for the
-    // credentials it inserted in.
-    final Text SECRET_KEY_ALIAS = new Text("secretkeyalias");
-    final byte[] SECRET_KEY = ("secretkey").getBytes();
-    Map<ApplicationAccessType, String> acls =
-        new HashMap<ApplicationAccessType, String>(1);
-    acls.put(ApplicationAccessType.VIEW_APP, "otheruser");
-    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
-    JobId jobId = MRBuilderUtils.newJobId(appId, 1);
-    TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
-    Path jobFile = mock(Path.class);
-
-    EventHandler eventHandler = mock(EventHandler.class);
-    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0));
-
-    JobConf jobConf = new JobConf();
-    jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
-    jobConf.setBoolean("fs.file.impl.disable.cache", true);
-    jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, "");
-
-    // setup UGI for security so tokens and keys are preserved
-    jobConf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION, "kerberos");
-    UserGroupInformation.setConfiguration(jobConf);
-
-    Credentials credentials = new Credentials();
-    credentials.addSecretKey(SECRET_KEY_ALIAS, SECRET_KEY);
-    Token<JobTokenIdentifier> jobToken = new Token<JobTokenIdentifier>(
-        ("tokenid").getBytes(), ("tokenpw").getBytes(),
-        new Text("tokenkind"), new Text("tokenservice"));
-    
-    TaskAttemptImpl taImpl =
-        new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
-            mock(TaskSplitMetaInfo.class), jobConf, taListener,
-            mock(OutputCommitter.class), jobToken, credentials,
-            new SystemClock(), null);
-
-    jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, taImpl.getID().toString());
-    ContainerId containerId = BuilderUtils.newContainerId(1, 1, 1, 1);
-    
-    ContainerLaunchContext launchCtx =
-        TaskAttemptImpl.createContainerLaunchContext(acls, containerId,
-            jobConf, jobToken, taImpl.createRemoteTask(),
-            TypeConverter.fromYarn(jobId), mock(Resource.class),
-            mock(WrappedJvmID.class), taListener,
-            credentials);
-
-    Assert.assertEquals("ACLs mismatch", acls, launchCtx.getApplicationACLs());
-    Credentials launchCredentials = new Credentials();
-
-    DataInputByteBuffer dibb = new DataInputByteBuffer();
-    dibb.reset(launchCtx.getContainerTokens());
-    launchCredentials.readTokenStorageStream(dibb);
-
-    // verify all tokens specified for the task attempt are in the launch context
-    for (Token<? extends TokenIdentifier> token : credentials.getAllTokens()) {
-      Token<? extends TokenIdentifier> launchToken =
-          launchCredentials.getToken(token.getService());
-      Assert.assertNotNull("Token " + token.getService() + " is missing",
-          launchToken);
-      Assert.assertEquals("Token " + token.getService() + " mismatch",
-          token, launchToken);
-    }
-
-    // verify the secret key is in the launch context
-    Assert.assertNotNull("Secret key missing",
-        launchCredentials.getSecretKey(SECRET_KEY_ALIAS));
-    Assert.assertTrue("Secret key mismatch", Arrays.equals(SECRET_KEY,
-        launchCredentials.getSecretKey(SECRET_KEY_ALIAS)));
-  }
 
   static public class StubbedFS extends RawLocalFileSystem {
     @Override
@@ -199,67 +119,66 @@ public class TestTaskAttempt{
 
   @Test
   public void testSingleRackRequest() throws Exception {
-    TaskAttemptImpl.RequestContainerTransition rct =
-        new TaskAttemptImpl.RequestContainerTransition(false);
+    TaskAttemptImpl.ScheduleTaskattempt sta = new TaskAttemptImpl.ScheduleTaskattempt();
 
     EventHandler eventHandler = mock(EventHandler.class);
     String[] hosts = new String[3];
     hosts[0] = "host1";
     hosts[1] = "host2";
     hosts[2] = "host3";
-    TaskSplitMetaInfo splitInfo =
-        new TaskSplitMetaInfo(hosts, 0, 128 * 1024 * 1024l);
+    TaskSplitMetaInfo splitInfo = new TaskSplitMetaInfo(hosts, 0,
+        128 * 1024 * 1024l);
 
-    TaskAttemptImpl mockTaskAttempt =
-        createMapTaskAttemptImplForTest(eventHandler, splitInfo);
-    TaskAttemptEvent mockTAEvent = mock(TaskAttemptEvent.class);
+    TaskAttemptImpl mockTaskAttempt = createMapTaskAttemptImpl2ForTest(
+        eventHandler, splitInfo);
+    TaskAttemptScheduleEvent mockTAEvent = mock(TaskAttemptScheduleEvent.class);
+    doReturn(false).when(mockTAEvent).isRescheduled();
 
-    rct.transition(mockTaskAttempt, mockTAEvent);
+    sta.transition(mockTaskAttempt, mockTAEvent);
 
     ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
     verify(eventHandler, times(2)).handle(arg.capture());
-    if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) {
+    if (!(arg.getAllValues().get(1) instanceof AMSchedulerTALaunchRequestEvent)) {
       Assert.fail("Second Event not of type ContainerRequestEvent");
     }
-    ContainerRequestEvent cre =
-        (ContainerRequestEvent) arg.getAllValues().get(1);
-    String[] requestedRacks = cre.getRacks();
-    //Only a single occurrence of /DefaultRack
+    AMSchedulerTALaunchRequestEvent tlrE = (AMSchedulerTALaunchRequestEvent) arg
+        .getAllValues().get(1);
+    String[] requestedRacks = tlrE.getRacks();
+    // Only a single occurrence of /DefaultRack
     assertEquals(1, requestedRacks.length);
   }
- 
+
   @Test
   public void testHostResolveAttempt() throws Exception {
-    TaskAttemptImpl.RequestContainerTransition rct =
-        new TaskAttemptImpl.RequestContainerTransition(false);
+    TaskAttemptImpl.ScheduleTaskattempt sta = new TaskAttemptImpl.ScheduleTaskattempt();
 
     EventHandler eventHandler = mock(EventHandler.class);
-    String[] hosts = new String[3];
-    hosts[0] = "192.168.1.1";
-    hosts[1] = "host2";
-    hosts[2] = "host3";
+    String hosts[] = new String[] {"192.168.1.1", "host2", "host3"};
+    String resolved[] = new String[] {"host1", "host2", "host3"};
     TaskSplitMetaInfo splitInfo =
         new TaskSplitMetaInfo(hosts, 0, 128 * 1024 * 1024l);
 
     TaskAttemptImpl mockTaskAttempt =
-        createMapTaskAttemptImplForTest(eventHandler, splitInfo);
+        createMapTaskAttemptImpl2ForTest(eventHandler, splitInfo);
     TaskAttemptImpl spyTa = spy(mockTaskAttempt);
-    when(spyTa.resolveHost(hosts[0])).thenReturn("host1");
+    when(spyTa.resolveHosts(hosts)).thenReturn(resolved);
+
+    TaskAttemptScheduleEvent mockTAEvent = mock(TaskAttemptScheduleEvent.class);
+    doReturn(false).when(mockTAEvent).isRescheduled();
 
-    TaskAttemptEvent mockTAEvent = mock(TaskAttemptEvent.class);
-    rct.transition(spyTa, mockTAEvent);
-    verify(spyTa).resolveHost(hosts[0]);
+    sta.transition(spyTa, mockTAEvent);
+    verify(spyTa).resolveHosts(hosts);
     ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
     verify(eventHandler, times(2)).handle(arg.capture());
-    if (!(arg.getAllValues().get(1) instanceof ContainerRequestEvent)) {
+    if (!(arg.getAllValues().get(1) instanceof AMSchedulerTALaunchRequestEvent)) {
       Assert.fail("Second Event not of type ContainerRequestEvent");
     }
     Map<String, Boolean> expected = new HashMap<String, Boolean>();
     expected.put("host1", true);
     expected.put("host2", true);
     expected.put("host3", true);
-    ContainerRequestEvent cre =
-        (ContainerRequestEvent) arg.getAllValues().get(1);
+    AMSchedulerTALaunchRequestEvent cre =
+        (AMSchedulerTALaunchRequestEvent) arg.getAllValues().get(1);
     String[] requestedHosts = cre.getHosts();
     for (String h : requestedHosts) {
       expected.remove(h);
@@ -326,13 +245,13 @@ public class TestTaskAttempt{
             .getValue());
   }
   
-  private TaskAttemptImpl createMapTaskAttemptImplForTest(
+  private TaskAttemptImpl createMapTaskAttemptImpl2ForTest(
       EventHandler eventHandler, TaskSplitMetaInfo taskSplitMetaInfo) {
     Clock clock = new SystemClock();
-    return createMapTaskAttemptImplForTest(eventHandler, taskSplitMetaInfo, clock);
+    return createMapTaskAttemptImpl2ForTest(eventHandler, taskSplitMetaInfo, clock);
   }
   
-  private TaskAttemptImpl createMapTaskAttemptImplForTest(
+  private TaskAttemptImpl createMapTaskAttemptImpl2ForTest(
       EventHandler eventHandler, TaskSplitMetaInfo taskSplitMetaInfo, Clock clock) {
     ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
@@ -342,9 +261,9 @@ public class TestTaskAttempt{
     JobConf jobConf = new JobConf();
     OutputCommitter outputCommitter = mock(OutputCommitter.class);
     TaskAttemptImpl taImpl =
-        new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
+        new MapTaskAttemptImpl2(taskId, 1, eventHandler, jobFile, 1,
             taskSplitMetaInfo, jobConf, taListener, outputCommitter, null,
-            null, clock, null);
+            null, clock, mock(TaskHeartbeatHandler.class), null);
     return taImpl;
   }
 
@@ -384,9 +303,10 @@ public class TestTaskAttempt{
           new TaskAttemptDiagnosticsUpdateEvent(attemptID,
               "Test Diagnostic Event"));
       getContext().getEventHandler().handle(
-          new TaskAttemptEvent(attemptID, TaskAttemptEventType.TA_FAILMSG));
+          new TaskAttemptEventFailRequest(attemptID, "Test Diagnostic Event"));
     }
 
+    // TODO XXX: This will execute in a separate thread. The assert is not very useful.
     protected EventHandler<JobHistoryEvent> createJobHistoryHandler(
         AppContext context) {
       return new EventHandler<JobHistoryEvent>() {
@@ -402,7 +322,7 @@ public class TestTaskAttempt{
       };
     }
   }
-  
+
   @Test
   public void testLaunchFailedWhileKilling() throws Exception {
     ApplicationId appId = BuilderUtils.newApplicationId(1, 2);
@@ -426,11 +346,14 @@ public class TestTaskAttempt{
     TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
     when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
     
+    AppContext mockAppContext = mock(AppContext.class);
+    doReturn(new ClusterInfo()).when(mockAppContext).getClusterInfo();
+    
     TaskAttemptImpl taImpl =
-      new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
+      new MapTaskAttemptImpl2(taskId, 1, eventHandler, jobFile, 1,
           splits, jobConf, taListener,
           mock(OutputCommitter.class), mock(Token.class), new Credentials(),
-          new SystemClock(), null);
+          new SystemClock(), mock(TaskHeartbeatHandler.class), mockAppContext);
 
     NodeId nid = BuilderUtils.newNodeId("127.0.0.1", 0);
     ContainerId contId = BuilderUtils.newContainerId(appAttemptId, 3);
@@ -438,24 +361,20 @@ public class TestTaskAttempt{
     when(container.getId()).thenReturn(contId);
     when(container.getNodeId()).thenReturn(nid);
     
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_SCHEDULE));
-    taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
-        container, mock(Map.class)));
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_KILL));
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_CONTAINER_CLEANED));
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_CONTAINER_LAUNCH_FAILED));
+    taImpl.handle(new TaskAttemptScheduleEvent(attemptId, false));
+    // At state STARTING.
+    taImpl.handle(new TaskAttemptEventKillRequest(attemptId, null));
+    // At some KILLING state.
+    taImpl.handle(new TaskAttemptEventContainerTerminating(attemptId, null));
     assertFalse(eventHandler.internalError);
   }
-  
+
+  // TODO Add a similar test for TERMINATING.
   @Test
-  public void testContainerCleanedWhileRunning() throws Exception {
+  public void testContainerTerminatedWhileRunning() throws Exception {
     ApplicationId appId = BuilderUtils.newApplicationId(1, 2);
-    ApplicationAttemptId appAttemptId =
-      BuilderUtils.newApplicationAttemptId(appId, 0);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 0);
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
     TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
@@ -463,7 +382,8 @@ public class TestTaskAttempt{
 
     MockEventHandler eventHandler = new MockEventHandler();
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0));
+    when(taListener.getAddress()).thenReturn(
+        new InetSocketAddress("localhost", 0));
 
     JobConf jobConf = new JobConf();
     jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -472,20 +392,7 @@ public class TestTaskAttempt{
     jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10");
 
     TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
-    when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
-
-    AppContext appCtx = mock(AppContext.class);
-    ClusterInfo clusterInfo = mock(ClusterInfo.class);
-    Resource resource = mock(Resource.class);
-    when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
-    when(clusterInfo.getMinContainerCapability()).thenReturn(resource);
-    when(resource.getMemory()).thenReturn(1024);
-
-    TaskAttemptImpl taImpl =
-      new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
-          splits, jobConf, taListener,
-          mock(OutputCommitter.class), mock(Token.class), new Credentials(),
-          new SystemClock(), appCtx);
+    when(splits.getLocations()).thenReturn(new String[] { "127.0.0.1" });
 
     NodeId nid = BuilderUtils.newNodeId("127.0.0.1", 0);
     ContainerId contId = BuilderUtils.newContainerId(appAttemptId, 3);
@@ -494,24 +401,36 @@ public class TestTaskAttempt{
     when(container.getNodeId()).thenReturn(nid);
     when(container.getNodeHttpAddress()).thenReturn("localhost:0");
 
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_SCHEDULE));
-    taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
-        container, mock(Map.class)));
-    taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0));
+    AppContext appCtx = mock(AppContext.class);
+    AMContainerMap containers = new AMContainerMap(
+        mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
+        appCtx);
+    containers.addContainerIfNew(container);
+
+    doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
+    doReturn(containers).when(appCtx).getAllContainers();
+
+    TaskAttemptImpl taImpl = new MapTaskAttemptImpl2(taskId, 1, eventHandler,
+        jobFile, 1, splits, jobConf, taListener, mock(OutputCommitter.class),
+        mock(Token.class), new Credentials(), new SystemClock(),
+        mock(TaskHeartbeatHandler.class), appCtx);
+
+    taImpl.handle(new TaskAttemptScheduleEvent(attemptId, false));
+    // At state STARTING.
+    taImpl.handle(new TaskAttemptRemoteStartEvent(attemptId, contId, null, -1));
     assertEquals("Task attempt is not in running state", taImpl.getState(),
         TaskAttemptState.RUNNING);
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_CONTAINER_CLEANED));
-    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+    taImpl.handle(new TaskAttemptEventContainerTerminated(attemptId, null));
+    assertFalse(
+        "InternalError occurred trying to handle TA_CONTAINER_TERMINATED",
         eventHandler.internalError);
   }
 
   @Test
-  public void testContainerCleanedWhileCommitting() throws Exception {
+  public void testContainerTerminatedWhileCommitting() throws Exception {
     ApplicationId appId = BuilderUtils.newApplicationId(1, 2);
-    ApplicationAttemptId appAttemptId =
-      BuilderUtils.newApplicationAttemptId(appId, 0);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 0);
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
     TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
@@ -519,7 +438,8 @@ public class TestTaskAttempt{
 
     MockEventHandler eventHandler = new MockEventHandler();
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0));
+    when(taListener.getAddress()).thenReturn(
+        new InetSocketAddress("localhost", 0));
 
     JobConf jobConf = new JobConf();
     jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -528,20 +448,7 @@ public class TestTaskAttempt{
     jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10");
 
     TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
-    when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
-
-    AppContext appCtx = mock(AppContext.class);
-    ClusterInfo clusterInfo = mock(ClusterInfo.class);
-    Resource resource = mock(Resource.class);
-    when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
-    when(clusterInfo.getMinContainerCapability()).thenReturn(resource);
-    when(resource.getMemory()).thenReturn(1024);
-
-    TaskAttemptImpl taImpl =
-      new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
-          splits, jobConf, taListener,
-          mock(OutputCommitter.class), mock(Token.class), new Credentials(),
-          new SystemClock(), appCtx);
+    when(splits.getLocations()).thenReturn(new String[] { "127.0.0.1" });
 
     NodeId nid = BuilderUtils.newNodeId("127.0.0.1", 0);
     ContainerId contId = BuilderUtils.newContainerId(appAttemptId, 3);
@@ -550,27 +457,40 @@ public class TestTaskAttempt{
     when(container.getNodeId()).thenReturn(nid);
     when(container.getNodeHttpAddress()).thenReturn("localhost:0");
 
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_SCHEDULE));
-    taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
-        container, mock(Map.class)));
-    taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0));
+    AppContext appCtx = mock(AppContext.class);
+    AMContainerMap containers = new AMContainerMap(
+        mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
+        appCtx);
+    containers.addContainerIfNew(container);
+
+    doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
+    doReturn(containers).when(appCtx).getAllContainers();
+
+    TaskAttemptImpl taImpl = new MapTaskAttemptImpl2(taskId, 1, eventHandler,
+        jobFile, 1, splits, jobConf, taListener, mock(OutputCommitter.class),
+        mock(Token.class), new Credentials(), new SystemClock(),
+        mock(TaskHeartbeatHandler.class), appCtx);
+
+    taImpl.handle(new TaskAttemptScheduleEvent(attemptId, false));
+    // At state STARTING.
+    taImpl.handle(new TaskAttemptRemoteStartEvent(attemptId, contId, null, -1));
+    assertEquals("Task attempt is not in running state", taImpl.getState(),
+        TaskAttemptState.RUNNING);
     taImpl.handle(new TaskAttemptEvent(attemptId,
         TaskAttemptEventType.TA_COMMIT_PENDING));
-
-    assertEquals("Task attempt is not in commit pending state", taImpl.getState(),
-        TaskAttemptState.COMMIT_PENDING);
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_CONTAINER_CLEANED));
-    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+    assertEquals("Task attempt is not in commit pending state",
+        taImpl.getState(), TaskAttemptState.COMMIT_PENDING);
+    taImpl.handle(new TaskAttemptEventContainerTerminated(attemptId, null));
+    assertFalse(
+        "InternalError occurred trying to handle TA_CONTAINER_TERMINATED",
         eventHandler.internalError);
   }
-  
+
   @Test
-  public void testDoubleTooManyFetchFailure() throws Exception {
+  public void testMultipleTooManyFetchFailures() throws Exception {
     ApplicationId appId = BuilderUtils.newApplicationId(1, 2);
-    ApplicationAttemptId appAttemptId =
-      BuilderUtils.newApplicationAttemptId(appId, 0);
+    ApplicationAttemptId appAttemptId = BuilderUtils.newApplicationAttemptId(
+        appId, 0);
     JobId jobId = MRBuilderUtils.newJobId(appId, 1);
     TaskId taskId = MRBuilderUtils.newTaskId(jobId, 1, TaskType.MAP);
     TaskAttemptId attemptId = MRBuilderUtils.newTaskAttemptId(taskId, 0);
@@ -578,7 +498,8 @@ public class TestTaskAttempt{
 
     MockEventHandler eventHandler = new MockEventHandler();
     TaskAttemptListener taListener = mock(TaskAttemptListener.class);
-    when(taListener.getAddress()).thenReturn(new InetSocketAddress("localhost", 0));
+    when(taListener.getAddress()).thenReturn(
+        new InetSocketAddress("localhost", 0));
 
     JobConf jobConf = new JobConf();
     jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
@@ -587,20 +508,7 @@ public class TestTaskAttempt{
     jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, "10");
 
     TaskSplitMetaInfo splits = mock(TaskSplitMetaInfo.class);
-    when(splits.getLocations()).thenReturn(new String[] {"127.0.0.1"});
-
-    AppContext appCtx = mock(AppContext.class);
-    ClusterInfo clusterInfo = mock(ClusterInfo.class);
-    Resource resource = mock(Resource.class);
-    when(appCtx.getClusterInfo()).thenReturn(clusterInfo);
-    when(clusterInfo.getMinContainerCapability()).thenReturn(resource);
-    when(resource.getMemory()).thenReturn(1024);
-
-    TaskAttemptImpl taImpl =
-      new MapTaskAttemptImpl(taskId, 1, eventHandler, jobFile, 1,
-          splits, jobConf, taListener,
-          mock(OutputCommitter.class), mock(Token.class), new Credentials(),
-          new SystemClock(), appCtx);
+    when(splits.getLocations()).thenReturn(new String[] { "127.0.0.1" });
 
     NodeId nid = BuilderUtils.newNodeId("127.0.0.1", 0);
     ContainerId contId = BuilderUtils.newContainerId(appAttemptId, 3);
@@ -609,27 +517,36 @@ public class TestTaskAttempt{
     when(container.getNodeId()).thenReturn(nid);
     when(container.getNodeHttpAddress()).thenReturn("localhost:0");
 
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_SCHEDULE));
-    taImpl.handle(new TaskAttemptContainerAssignedEvent(attemptId,
-        container, mock(Map.class)));
-    taImpl.handle(new TaskAttemptContainerLaunchedEvent(attemptId, 0));
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_DONE));
-    taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_CONTAINER_CLEANED));
-    
+    AppContext appCtx = mock(AppContext.class);
+    AMContainerMap containers = new AMContainerMap(
+        mock(ContainerHeartbeatHandler.class), mock(TaskAttemptListener.class),
+        appCtx);
+    containers.addContainerIfNew(container);
+
+    doReturn(new ClusterInfo()).when(appCtx).getClusterInfo();
+    doReturn(containers).when(appCtx).getAllContainers();
+
+    TaskAttemptImpl taImpl = new MapTaskAttemptImpl2(taskId, 1, eventHandler,
+        jobFile, 1, splits, jobConf, taListener, mock(OutputCommitter.class),
+        mock(Token.class), new Credentials(), new SystemClock(),
+        mock(TaskHeartbeatHandler.class), appCtx);
+
+    taImpl.handle(new TaskAttemptScheduleEvent(attemptId, false));
+    // At state STARTING.
+    taImpl.handle(new TaskAttemptRemoteStartEvent(attemptId, contId, null, -1));
+    taImpl
+        .handle(new TaskAttemptEvent(attemptId, TaskAttemptEventType.TA_DONE));
     assertEquals("Task attempt is not in succeeded state", taImpl.getState(),
         TaskAttemptState.SUCCEEDED);
     taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
+        TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES));
     assertEquals("Task attempt is not in FAILED state", taImpl.getState(),
         TaskAttemptState.FAILED);
     taImpl.handle(new TaskAttemptEvent(attemptId,
-        TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURE));
-    assertEquals("Task attempt is not in FAILED state, still", taImpl.getState(),
-        TaskAttemptState.FAILED);
-    assertFalse("InternalError occurred trying to handle TA_CONTAINER_CLEANED",
+        TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES));
+    assertEquals("Task attempt is not in FAILED state, still",
+        taImpl.getState(), TaskAttemptState.FAILED);
+    assertFalse("InternalError occurred trying to handle TA_TOO_MANY_FETCH_FAILURES",
         eventHandler.internalError);
   }
 

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/TestRMContainerAllocator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/TestRMContainerAllocator.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/TestRMContainerAllocator.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/TestRMContainerAllocator.java Fri Oct 19 19:01:10 2012
@@ -71,6 +71,7 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.net.NetworkTopology;
 import org.apache.hadoop.yarn.Clock;
+import org.apache.hadoop.yarn.ClusterInfo;
 import org.apache.hadoop.yarn.SystemClock;
 import org.apache.hadoop.yarn.api.AMRMProtocol;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
@@ -801,7 +802,7 @@ public class TestRMContainerAllocator {
   
   static class RMContainerAllocatorForTest extends RMContainerAllocator {
 
-    public RMContainerAllocatorForTest(RMContainerRequestor requestor,
+    public RMContainerAllocatorForTest(ContainerRequestor requestor,
         AppContext appContext) {
       super(requestor, appContext);
     }
@@ -840,7 +841,7 @@ public class TestRMContainerAllocator {
 
     boolean recalculatedReduceSchedule = false;
 
-    public RecalculateContainerAllocator(RMContainerRequestor requestor,
+    public RecalculateContainerAllocator(ContainerRequestor requestor,
         AppContext appContext) {
       super(requestor, appContext);
     }
@@ -899,7 +900,7 @@ public class TestRMContainerAllocator {
     }
     
     @Override
-    protected Resource getAvailableResources() {
+    public Resource getAvailableResources() {
       return BuilderUtils.newResource(0);
     }
     
@@ -995,7 +996,13 @@ public class TestRMContainerAllocator {
     when(appContext.getJob(jobId)).thenReturn(mockJob);
     when(appContext.getClock()).thenReturn(clock);
     when(appContext.getAllNodes()).thenReturn(amNodeMap);
+    when(appContext.getClusterInfo()).thenReturn(
+        new ClusterInfo(BuilderUtils.newResource(1024), BuilderUtils
+            .newResource(10240)));
 
     return appContext;
   }
+  
+  // TODO Add a unit test to verify a correct launchContainer invocation with
+  // security.
 }

Added: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/TestAMContainerHelpers.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/TestAMContainerHelpers.java?rev=1400232&view=auto
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/TestAMContainerHelpers.java (added)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/TestAMContainerHelpers.java Fri Oct 19 19:01:10 2012
@@ -0,0 +1,141 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.mapreduce.v2.app2.rm.container;
+
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Map;
+
+import junit.framework.Assert;
+
+import org.apache.hadoop.fs.CommonConfigurationKeysPublic;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
+import org.apache.hadoop.io.DataInputByteBuffer;
+import org.apache.hadoop.io.Text;
+import org.apache.hadoop.mapred.JobConf;
+import org.apache.hadoop.mapred.WrappedJvmID;
+import org.apache.hadoop.mapreduce.MRJobConfig;
+import org.apache.hadoop.mapreduce.TypeConverter;
+import org.apache.hadoop.mapreduce.security.token.JobTokenIdentifier;
+import org.apache.hadoop.mapreduce.v2.api.records.JobId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskType;
+import org.apache.hadoop.mapreduce.v2.app2.TaskAttemptListener;
+import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.security.token.TokenIdentifier;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerLaunchContext;
+import org.apache.hadoop.yarn.api.records.Resource;
+import org.apache.hadoop.yarn.util.BuilderUtils;
+import org.junit.Test;
+
+public class TestAMContainerHelpers {
+
+  // WARNING: This test must be the only test in this file. This is because
+  // there is an optimization where the credentials passed in are cached
+  // statically so they do not need to be recomputed when creating a new
+  // ContainerLaunchContext. if other tests run first this code will cache
+  // their credentials and this test will fail trying to look for the
+  // credentials it inserted in.
+
+  @Test
+  public void testCLCConstruction() throws Exception {
+    final Text SECRET_KEY_ALIAS = new Text("secretkeyalias");
+    final byte[] SECRET_KEY = ("secretkey").getBytes();
+    Map<ApplicationAccessType, String> acls = new HashMap<ApplicationAccessType, String>(
+        1);
+    acls.put(ApplicationAccessType.VIEW_APP, "otheruser");
+    ApplicationId appId = BuilderUtils.newApplicationId(1, 1);
+    JobId jobId = MRBuilderUtils.newJobId(appId, 1);
+
+    TaskAttemptListener taListener = mock(TaskAttemptListener.class);
+    when(taListener.getAddress()).thenReturn(
+        new InetSocketAddress("localhost", 0));
+
+    JobConf jobConf = new JobConf();
+    jobConf.setClass("fs.file.impl", StubbedFS.class, FileSystem.class);
+    jobConf.setBoolean("fs.file.impl.disable.cache", true);
+    jobConf.set(JobConf.MAPRED_MAP_TASK_ENV, "");
+
+    // setup UGI for security so tokens and keys are preserved
+    jobConf.set(CommonConfigurationKeysPublic.HADOOP_SECURITY_AUTHENTICATION,
+        "kerberos");
+    UserGroupInformation.setConfiguration(jobConf);
+
+    Credentials credentials = new Credentials();
+    credentials.addSecretKey(SECRET_KEY_ALIAS, SECRET_KEY);
+    Token<JobTokenIdentifier> jobToken = new Token<JobTokenIdentifier>(
+        ("tokenid").getBytes(), ("tokenpw").getBytes(), new Text("tokenkind"),
+        new Text("tokenservice"));
+
+    jobConf.set(MRJobConfig.APPLICATION_ATTEMPT_ID, BuilderUtils
+        .newApplicationAttemptId(appId, 1).toString());
+    ContainerId containerId = BuilderUtils.newContainerId(1, 1, 1, 1);
+
+    ContainerLaunchContext launchCtx = AMContainerHelpers
+        .createContainerLaunchContext(acls, containerId, jobConf, TaskType.MAP,
+            jobToken, TypeConverter.fromYarn(jobId), mock(Resource.class),
+            mock(WrappedJvmID.class), taListener, credentials, false);
+
+    Assert.assertEquals("ACLs mismatch", acls, launchCtx.getApplicationACLs());
+    Credentials launchCredentials = new Credentials();
+
+    DataInputByteBuffer dibb = new DataInputByteBuffer();
+    dibb.reset(launchCtx.getContainerTokens());
+    launchCredentials.readTokenStorageStream(dibb);
+
+    // verify all tokens specified for the task attempt are in the launch
+    // context
+    for (Token<? extends TokenIdentifier> token : credentials.getAllTokens()) {
+      Token<? extends TokenIdentifier> launchToken = launchCredentials
+          .getToken(token.getService());
+      Assert.assertNotNull("Token " + token.getService() + " is missing",
+          launchToken);
+      Assert.assertEquals("Token " + token.getService() + " mismatch", token,
+          launchToken);
+    }
+
+    // verify the secret key is in the launch context
+    Assert.assertNotNull("Secret key missing",
+        launchCredentials.getSecretKey(SECRET_KEY_ALIAS));
+    Assert.assertTrue(
+        "Secret key mismatch",
+        Arrays.equals(SECRET_KEY,
+            launchCredentials.getSecretKey(SECRET_KEY_ALIAS)));
+  }
+
+  static public class StubbedFS extends RawLocalFileSystem {
+    @Override
+    public FileStatus getFileStatus(Path f) throws IOException {
+      return new FileStatus(1, false, 1, 1, 1, f);
+    }
+  }
+}
\ No newline at end of file

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebApp.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebApp.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebApp.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebApp.java Fri Oct 19 19:01:10 2012
@@ -32,9 +32,12 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
 import org.apache.hadoop.mapreduce.v2.app2.job.Task;
 import org.apache.hadoop.mapreduce.v2.app2.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerMap;
+import org.apache.hadoop.mapreduce.v2.app2.rm.node.AMNodeMap;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -113,6 +116,21 @@ public class TestAMWebApp {
     public ClusterInfo getClusterInfo() {
       return null;
     }
+
+    @Override
+    public Map<ApplicationAccessType, String> getApplicationACLs() {
+      return null;
+    }
+
+    @Override
+    public AMContainerMap getAllContainers() {
+      return null;
+    }
+
+    @Override
+    public AMNodeMap getAllNodes() {
+      return null;
+    }
   }
 
   @Test public void testAppControllerIndex() {

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServices.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServices.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServices.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServices.java Fri Oct 19 19:01:10 2012
@@ -34,8 +34,11 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.AppContext;
 import org.apache.hadoop.mapreduce.v2.app2.MockJobs;
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerMap;
+import org.apache.hadoop.mapreduce.v2.app2.rm.node.AMNodeMap;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -141,6 +144,21 @@ public class TestAMWebServices extends J
     public ClusterInfo getClusterInfo() {
       return null;
     }
+
+    @Override
+    public Map<ApplicationAccessType, String> getApplicationACLs() {
+      return null;
+    }
+
+    @Override
+    public AMContainerMap getAllContainers() {
+      return null;
+    }
+
+    @Override
+    public AMNodeMap getAllNodes() {
+      return null;
+    }
   }
 
   private Injector injector = Guice.createInjector(new ServletModule() {

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesAttempts.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesAttempts.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesAttempts.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesAttempts.java Fri Oct 19 19:01:10 2012
@@ -40,9 +40,12 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
 import org.apache.hadoop.mapreduce.v2.app2.job.Task;
 import org.apache.hadoop.mapreduce.v2.app2.job.TaskAttempt;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerMap;
+import org.apache.hadoop.mapreduce.v2.app2.rm.node.AMNodeMap;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -151,6 +154,21 @@ public class TestAMWebServicesAttempts e
     public ClusterInfo getClusterInfo() {
       return null;
     }
+
+    @Override
+    public Map<ApplicationAccessType, String> getApplicationACLs() {
+      return null;
+    }
+
+    @Override
+    public AMContainerMap getAllContainers() {
+      return null;
+    }
+
+    @Override
+    public AMNodeMap getAllNodes() {
+      return null;
+    }
   }
 
   private Injector injector = Guice.createInjector(new ServletModule() {

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobConf.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobConf.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobConf.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobConf.java Fri Oct 19 19:01:10 2012
@@ -42,9 +42,12 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.AppContext;
 import org.apache.hadoop.mapreduce.v2.app2.MockJobs;
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerMap;
+import org.apache.hadoop.mapreduce.v2.app2.rm.node.AMNodeMap;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -152,6 +155,21 @@ public class TestAMWebServicesJobConf ex
     public ClusterInfo getClusterInfo() {
       return null;
     }
+
+    @Override
+    public Map<ApplicationAccessType, String> getApplicationACLs() {
+      return null;
+    }
+
+    @Override
+    public AMContainerMap getAllContainers() {
+      return null;
+    }
+
+    @Override
+    public AMNodeMap getAllNodes() {
+      return null;
+    }
   }
 
   private Injector injector = Guice.createInjector(new ServletModule() {

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobs.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobs.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobs.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesJobs.java Fri Oct 19 19:01:10 2012
@@ -40,10 +40,13 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.AppContext;
 import org.apache.hadoop.mapreduce.v2.app2.MockJobs;
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerMap;
+import org.apache.hadoop.mapreduce.v2.app2.rm.node.AMNodeMap;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.security.authorize.AccessControlList;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -154,6 +157,21 @@ public class TestAMWebServicesJobs exten
     public ClusterInfo getClusterInfo() {
       return null;
     }
+
+    @Override
+    public Map<ApplicationAccessType, String> getApplicationACLs() {
+      return null;
+    }
+
+    @Override
+    public AMContainerMap getAllContainers() {
+      return null;
+    }
+
+    @Override
+    public AMNodeMap getAllNodes() {
+      return null;
+    }
   }
 
   private Injector injector = Guice.createInjector(new ServletModule() {

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesTasks.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesTasks.java?rev=1400232&r1=1400231&r2=1400232&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesTasks.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/webapp/TestAMWebServicesTasks.java Fri Oct 19 19:01:10 2012
@@ -38,9 +38,12 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.MockJobs;
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
 import org.apache.hadoop.mapreduce.v2.app2.job.Task;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerMap;
+import org.apache.hadoop.mapreduce.v2.app2.rm.node.AMNodeMap;
 import org.apache.hadoop.mapreduce.v2.util.MRApps;
 import org.apache.hadoop.yarn.Clock;
 import org.apache.hadoop.yarn.ClusterInfo;
+import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
@@ -148,6 +151,21 @@ public class TestAMWebServicesTasks exte
     public ClusterInfo getClusterInfo() {
       return null;
     }
+
+    @Override
+    public Map<ApplicationAccessType, String> getApplicationACLs() {
+      return null;
+    }
+
+    @Override
+    public AMContainerMap getAllContainers() {
+      return null;
+    }
+
+    @Override
+    public AMNodeMap getAllNodes() {
+      return null;
+    }
   }
 
   private Injector injector = Guice.createInjector(new ServletModule() {