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 20:52:09 UTC

svn commit: r1400225 [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/job/ hadoop-mapreduce-client/hadoop-mapreduce-client-app...

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/TestFail.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/TestFail.java?rev=1400225&r1=1400224&r2=1400225&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/TestFail.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/TestFail.java Fri Oct 19 18:52:09 2012
@@ -37,8 +37,10 @@ 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.job.TaskAttemptStateInternal;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventType;
+import org.apache.hadoop.mapreduce.v2.app2.job.impl.TaskAttemptImpl;
 import org.apache.hadoop.mapreduce.v2.app2.launcher.ContainerLauncher;
 import org.apache.hadoop.mapreduce.v2.app2.launcher.ContainerLauncherImpl;
 import org.apache.hadoop.mapreduce.v2.app2.rm.NMCommunicatorEvent;
@@ -191,7 +193,8 @@ public class TestFail {
     Assert.assertEquals("Num attempts is not correct", maxAttempts, attempts
         .size());
     TaskAttempt attempt = attempts.values().iterator().next();
-    app.waitForState(attempt, TaskAttemptState.START_WAIT);
+    app.waitForInternalState((TaskAttemptImpl) attempt,
+        TaskAttemptStateInternal.START_WAIT);
     // TODO XXX: This may not be a valid test.
     app.getDispatcher().getEventHandler().handle(
         new TaskAttemptEvent(attempt.getID(),

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/TestJobImpl.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/TestJobImpl.java?rev=1400225&r1=1400224&r2=1400225&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/TestJobImpl.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/TestJobImpl.java Fri Oct 19 18:52:09 2012
@@ -46,6 +46,7 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.app2.MRApp;
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
+import org.apache.hadoop.mapreduce.v2.app2.job.JobStateInternal;
 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;
@@ -81,11 +82,11 @@ public class TestJobImpl {
     tasks.put(mockTask.getID(), mockTask);
     mockJob.tasks = tasks;
 
-    when(mockJob.getState()).thenReturn(JobState.ERROR);
+    when(mockJob.getInternalState()).thenReturn(JobStateInternal.ERROR);
     JobEvent mockJobEvent = mock(JobEvent.class);
-    JobState state = trans.transition(mockJob, mockJobEvent);
+    JobStateInternal state = trans.transition(mockJob, mockJobEvent);
     Assert.assertEquals("Incorrect state returned from JobNoTasksCompletedTransition",
-        JobState.ERROR, state);
+        JobStateInternal.ERROR, state);
   }
 
   @Test
@@ -102,9 +103,9 @@ public class TestJobImpl {
     when(mockJob.getJobContext()).thenReturn(mockJobContext);
     doNothing().when(mockJob).setFinishTime();
     doNothing().when(mockJob).logJobHistoryFinishedEvent();
-    when(mockJob.finished(JobState.KILLED)).thenReturn(JobState.KILLED);
-    when(mockJob.finished(JobState.FAILED)).thenReturn(JobState.FAILED);
-    when(mockJob.finished(JobState.SUCCEEDED)).thenReturn(JobState.SUCCEEDED);
+    when(mockJob.finished(JobStateInternal.KILLED)).thenReturn(JobStateInternal.KILLED);
+    when(mockJob.finished(JobStateInternal.FAILED)).thenReturn(JobStateInternal.FAILED);
+    when(mockJob.finished(JobStateInternal.SUCCEEDED)).thenReturn(JobStateInternal.SUCCEEDED);
 
     try {
       doThrow(new IOException()).when(mockCommitter).commitJob(any(JobContext.class));
@@ -116,7 +117,7 @@ public class TestJobImpl {
       "for successful job",
       JobImpl.checkJobCompleteSuccess(mockJob));
     Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
-        JobState.FAILED, JobImpl.checkJobCompleteSuccess(mockJob));
+        JobStateInternal.FAILED, JobImpl.checkJobCompleteSuccess(mockJob));
   }
 
   @Test
@@ -133,7 +134,7 @@ public class TestJobImpl {
     when(mockJob.getJobContext()).thenReturn(mockJobContext);
     doNothing().when(mockJob).setFinishTime();
     doNothing().when(mockJob).logJobHistoryFinishedEvent();
-    when(mockJob.finished(any(JobState.class))).thenReturn(JobState.SUCCEEDED);
+    when(mockJob.finished(any(JobStateInternal.class))).thenReturn(JobStateInternal.SUCCEEDED);
 
     try {
       doNothing().when(mockCommitter).commitJob(any(JobContext.class));
@@ -145,7 +146,7 @@ public class TestJobImpl {
       "for successful job",
       JobImpl.checkJobCompleteSuccess(mockJob));
     Assert.assertEquals("checkJobCompleteSuccess returns incorrect state",
-        JobState.SUCCEEDED, JobImpl.checkJobCompleteSuccess(mockJob));
+        JobStateInternal.SUCCEEDED, JobImpl.checkJobCompleteSuccess(mockJob));
   }
 
   @Test

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/TestTaskImpl.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/TestTaskImpl.java?rev=1400225&r1=1400224&r2=1400225&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/TestTaskImpl.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/TestTaskImpl.java Fri Oct 19 18:52:09 2012
@@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.AppContext;
 import org.apache.hadoop.mapreduce.v2.app2.TaskAttemptListener;
 import org.apache.hadoop.mapreduce.v2.app2.TaskHeartbeatHandler;
+import org.apache.hadoop.mapreduce.v2.app2.job.TaskStateInternal;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskEvent;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskTAttemptEvent;
@@ -333,7 +334,7 @@ public class TestTaskImpl {
    * {@link TaskState#KILL_WAIT}
    */
   private void assertTaskKillWaitState() {
-    assertEquals(TaskState.KILL_WAIT, mockTask.getState());
+    assertEquals(TaskStateInternal.KILL_WAIT, mockTask.getInternalState());
   }
   
   /**

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/launcher/TestContainerLauncher.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/launcher/TestContainerLauncher.java?rev=1400225&r1=1400224&r2=1400225&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/launcher/TestContainerLauncher.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/launcher/TestContainerLauncher.java Fri Oct 19 18:52:09 2012
@@ -37,7 +37,6 @@ import org.apache.hadoop.ipc.Server;
 import org.apache.hadoop.mapreduce.MRJobConfig;
 import org.apache.hadoop.mapreduce.v2.api.records.JobState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
-import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskId;
 import org.apache.hadoop.mapreduce.v2.api.records.TaskState;
 import org.apache.hadoop.mapreduce.v2.app2.AppContext;
@@ -45,6 +44,8 @@ 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.job.TaskAttemptStateInternal;
+import org.apache.hadoop.mapreduce.v2.app2.job.impl.TaskAttemptImpl;
 import org.apache.hadoop.mapreduce.v2.app2.rm.NMCommunicatorEvent;
 import org.apache.hadoop.mapreduce.v2.app2.rm.NMCommunicatorEventType;
 import org.apache.hadoop.mapreduce.v2.app2.rm.node.AMNodeMap;
@@ -263,7 +264,8 @@ public class TestContainerLauncher {
           attempts.size());
 
     TaskAttempt attempt = attempts.values().iterator().next();
-    app.waitForState(attempt, TaskAttemptState.START_WAIT);
+    app.waitForInternalState((TaskAttemptImpl) attempt,
+        TaskAttemptStateInternal.START_WAIT);
 
     app.waitForState(job, JobState.FAILED);