hadoop-mapreduce-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From ss...@apache.org
Subject svn commit: r1398523 [1/2] - in /hadoop/common/branches/MR-3902/hadoop-mapreduce-project: ./ hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapred/ hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java...
Date Mon, 15 Oct 2012 21:10:01 GMT
Author: sseth
Date: Mon Oct 15 21:09:59 2012
New Revision: 1398523

URL: http://svn.apache.org/viewvc?rev=1398523&view=rev
Log:
MAPREDUCE-4727. Handle successful NM stop requests. (sseth)

Added:
    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/event/TaskAttemptEventContainerTerminated.java
    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/event/TaskAttemptEventContainerTerminating.java
    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/event/TaskAttemptEventNodeFailed.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventTAEnded.java
Removed:
    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/event/TaskAttemptEventTerminated.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerTAStopRequestEvent.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerTASucceededEvent.java
Modified:
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/CHANGES.txt.MR-3902
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapred/LocalContainerAllocator.java
    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/event/TaskAttemptEventType.java
    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/impl/TaskAttemptImpl.java
    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/impl/TaskImpl.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/launcher/ContainerLauncherImpl.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/recover/RecoveryService.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventType.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/RMContainerAllocator.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEvent.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventStopFailed.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventType.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerImpl.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerLaunchRequestEvent.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerState.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/node/AMNodeEventTaskAttemptEnded.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/node/AMNodeEventType.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/node/AMNodeImpl.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/jobhistory/TestJobHistoryEventHandler.java
    hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/test/java/org/apache/hadoop/mapreduce/v2/app2/MRApp.java
    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
    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/TestMapReduceChildJVM.java
    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

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/CHANGES.txt.MR-3902
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/CHANGES.txt.MR-3902?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/CHANGES.txt.MR-3902 (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/CHANGES.txt.MR-3902 Mon Oct 15 21:09:59 2012
@@ -26,3 +26,5 @@ Branch MR-3902
   MAPREDUCE-4664. ContainerHeartbeatHandler should be pinged on a getTask call (sseth)
 
   MAPREDUCE-4663. Container Launch should be independent of o.a.h.m.Task (sseth)
+
+  MAPREDUCE-4727. Handle successful NM stop requests. (sseth)

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapred/LocalContainerAllocator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapred/LocalContainerAllocator.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapred/LocalContainerAllocator.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapred/LocalContainerAllocator.java Mon Oct 15 21:09:59 2012
@@ -47,12 +47,11 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.job.event.JobCounterUpdateEvent;
 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.TaskAttemptEventTerminated;
+import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventContainerTerminated;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptRemoteStartEvent;
 import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerEvent;
 import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTALaunchRequestEvent;
-import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTAStopRequestEvent;
-import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTASucceededEvent;
+import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerEventTAEnded;
 import org.apache.hadoop.mapreduce.v2.app2.rm.ContainerAllocator;
 import org.apache.hadoop.mapreduce.v2.app2.rm.RMCommunicator;
 import org.apache.hadoop.util.StringUtils;
@@ -290,7 +289,7 @@ public class LocalContainerAllocator ext
 
       // CLEANUP event generated.f
       appContext.getEventHandler().handle(
-          new TaskAttemptEventTerminated(attemptID));
+          new TaskAttemptEventContainerTerminated(attemptID, null));
 
     } catch (IOException ioe) {
       // if umbilical itself barfs (in error-handler of runSubMap()),
@@ -303,23 +302,23 @@ public class LocalContainerAllocator ext
   }
   
   @SuppressWarnings("unchecked")
-  public void handleTaStopRequest(AMSchedulerTAStopRequestEvent sEvent) {
+  public void handleTaStopRequest(AMSchedulerEventTAEnded sEvent) {
     // Implies a failed or killed task.
     // This will trigger a CLEANUP event. UberAM is supposed to fail if there's
     // event a single failed attempt. Hence the CLEANUP is OK (otherwise delay
     // cleanup till end of job). TODO Enforce job failure on single task attempt
     // failure.
     appContext.getEventHandler().handle(
-        new TaskAttemptEventTerminated(sEvent.getAttemptID()));
+        new TaskAttemptEventContainerTerminated(sEvent.getAttemptID(), null));
     taskAttemptListenern.unregisterTaskAttempt(sEvent.getAttemptID());
   }
 
   @SuppressWarnings("unchecked")
-  public void handleTaSucceededRequest(AMSchedulerTASucceededEvent sEvent) {
+  public void handleTaSucceededRequest(AMSchedulerEventTAEnded sEvent) {
     // Successful taskAttempt.
     // Same CLEANUP comment as handleTaStopRequest
     appContext.getEventHandler().handle(
-        new TaskAttemptEventTerminated(sEvent.getAttemptID()));
+        new TaskAttemptEventContainerTerminated(sEvent.getAttemptID(), null));
     taskAttemptListenern.unregisterTaskAttempt(sEvent.getAttemptID());
   }
 
@@ -329,11 +328,19 @@ public class LocalContainerAllocator ext
     case S_TA_LAUNCH_REQUEST:
       handleTaLaunchRequest((AMSchedulerTALaunchRequestEvent) sEvent);
       break;
-    case S_TA_STOP_REQUEST: // Effectively means a failure.
-      handleTaStopRequest((AMSchedulerTAStopRequestEvent) sEvent);
-      break;
-    case S_TA_SUCCEEDED:
-      handleTaSucceededRequest((AMSchedulerTASucceededEvent) sEvent);
+    case S_TA_ENDED: // Effectively means a failure.
+      AMSchedulerEventTAEnded event = (AMSchedulerEventTAEnded) sEvent;
+      switch(event.getState()) {
+      case FAILED:
+      case KILLED:
+        handleTaStopRequest(event);
+        break;
+      case SUCCEEDED:
+        handleTaSucceededRequest(event);
+        break;
+      default:
+        throw new YarnException("Unexpected TaskAttemptState: " + event.getState());
+      }
       break;
     default:
       LOG.warn("Invalid event type for LocalContainerAllocator: "

Added: 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/event/TaskAttemptEventContainerTerminated.java
URL: http://svn.apache.org/viewvc/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/event/TaskAttemptEventContainerTerminated.java?rev=1398523&view=auto
==============================================================================
--- 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/event/TaskAttemptEventContainerTerminated.java (added)
+++ 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/event/TaskAttemptEventContainerTerminated.java Mon Oct 15 21:09:59 2012
@@ -0,0 +1,34 @@
+/**
+ * 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.job.event;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+
+public class TaskAttemptEventContainerTerminated extends TaskAttemptEvent {
+
+  private final String message;
+
+  public TaskAttemptEventContainerTerminated(TaskAttemptId id, String message) {
+    super(id, TaskAttemptEventType.TA_CONTAINER_TERMINATED);
+    this.message = message;
+  }
+
+  public String getDiagnosticInfo() {
+    return message;
+  }
+}

Added: 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/event/TaskAttemptEventContainerTerminating.java
URL: http://svn.apache.org/viewvc/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/event/TaskAttemptEventContainerTerminating.java?rev=1398523&view=auto
==============================================================================
--- 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/event/TaskAttemptEventContainerTerminating.java (added)
+++ 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/event/TaskAttemptEventContainerTerminating.java Mon Oct 15 21:09:59 2012
@@ -0,0 +1,36 @@
+/**
+ * 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.job.event;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+
+public class TaskAttemptEventContainerTerminating extends TaskAttemptEvent {
+
+  private final String message;
+
+  public TaskAttemptEventContainerTerminating(TaskAttemptId id,
+      String diagMessage) {
+    super(id, TaskAttemptEventType.TA_CONTAINER_TERMINATING);
+    this.message = diagMessage;
+  }
+
+  public String getDiagnosticInfo() {
+    return this.message;
+  }
+
+}
\ No newline at end of file

Added: 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/event/TaskAttemptEventNodeFailed.java
URL: http://svn.apache.org/viewvc/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/event/TaskAttemptEventNodeFailed.java?rev=1398523&view=auto
==============================================================================
--- 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/event/TaskAttemptEventNodeFailed.java (added)
+++ 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/event/TaskAttemptEventNodeFailed.java Mon Oct 15 21:09:59 2012
@@ -0,0 +1,36 @@
+/**
+ * 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.job.event;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+
+public class TaskAttemptEventNodeFailed extends TaskAttemptEvent {
+
+  private final String message;
+
+  public TaskAttemptEventNodeFailed(TaskAttemptId id,
+      String diagMessage) {
+    super(id, TaskAttemptEventType.TA_NODE_FAILED);
+    this.message = diagMessage;
+  }
+
+  public String getDiagnosticInfo() {
+    return this.message;
+  }
+
+}
\ No newline at end of file

Modified: 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/event/TaskAttemptEventType.java
URL: http://svn.apache.org/viewvc/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/event/TaskAttemptEventType.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- 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/event/TaskAttemptEventType.java (original)
+++ 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/event/TaskAttemptEventType.java Mon Oct 15 21:09:59 2012
@@ -44,7 +44,9 @@ public enum TaskAttemptEventType {
   //Producer: Container / Scheduler.
   // Indicates that the RM considers the container to be complete. Implies the 
   // JVM is done, except in one case. TOOD: document the case.
-  TA_TERMINATED,
+  TA_CONTAINER_TERMINATING,
+  TA_CONTAINER_TERMINATED,
+  TA_NODE_FAILED,
   
   //Producer: Job
   TA_TOO_MANY_FETCH_FAILURES,

Modified: 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/impl/TaskAttemptImpl.java
URL: http://svn.apache.org/viewvc/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/impl/TaskAttemptImpl.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- 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/impl/TaskAttemptImpl.java (original)
+++ 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/impl/TaskAttemptImpl.java Mon Oct 15 21:09:59 2012
@@ -66,6 +66,9 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.job.event.JobTaskAttemptFetchFailureEvent;
 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.TaskAttemptEventNodeFailed;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptEventType;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptRemoteStartEvent;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskAttemptScheduleEvent;
@@ -74,8 +77,7 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskTAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTALaunchRequestEvent;
-import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTAStopRequestEvent;
-import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTASucceededEvent;
+import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerEventTAEnded;
 import org.apache.hadoop.mapreduce.v2.app2.speculate.SpeculatorEvent;
 import org.apache.hadoop.mapreduce.v2.app2.taskclean.TaskCleanupEvent;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
@@ -174,7 +176,9 @@ public abstract class TaskAttemptImpl im
         .addTransition(TaskAttemptState.START_WAIT, TaskAttemptState.START_WAIT, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
         .addTransition(TaskAttemptState.START_WAIT, TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_FAIL_REQUEST, createFailRequestBeforeRunningTransition())
         .addTransition(TaskAttemptState.START_WAIT, TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptEventType.TA_KILL_REQUEST, createKillRequestBeforeRunningTransition())
-        .addTransition(TaskAttemptState.START_WAIT, TaskAttemptState.FAILED, TaskAttemptEventType.TA_TERMINATED, createContainerCompletedBeforeRunningTransition())
+        .addTransition(TaskAttemptState.START_WAIT, TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptEventType.TA_NODE_FAILED, createNodeFailedBeforeRunningTransition())
+        .addTransition(TaskAttemptState.START_WAIT, TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_CONTAINER_TERMINATING, createContainerTerminatingBeforeRunningTransition())
+        .addTransition(TaskAttemptState.START_WAIT, TaskAttemptState.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, createContainerCompletedBeforeRunningTransition())
         
         .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.RUNNING, TaskAttemptEventType.TA_STATUS_UPDATE, createStatusUpdateTransition())
         .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.RUNNING, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
@@ -184,7 +188,11 @@ public abstract class TaskAttemptImpl im
         .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TIMED_OUT, createFailRequestWhileRunningTransition())
         .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_FAIL_REQUEST, createFailRequestWhileRunningTransition())
         .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptEventType.TA_KILL_REQUEST, createKillRequestWhileRunningTransition())
-        .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.FAILED, TaskAttemptEventType.TA_TERMINATED, createContainerCompletedWhileRunningTransition()) 
+        .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptEventType.TA_NODE_FAILED, createNodeFailedWhileRunningTransition())
+        .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.RUNNING, TaskAttemptEventType.TA_CONTAINER_TERMINATING, createContainerTerminatingWhileRunningTransition())
+        .addTransition(TaskAttemptState.RUNNING, TaskAttemptState.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, createContainerCompletedWhileRunningTransition())
+        
+        // XXX Maybe move getMessage / getDiagnosticInfo into the base TaskAttemptEvent ?
         
         .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.COMMIT_PENDING, TaskAttemptEventType.TA_STATUS_UPDATE, createStatusUpdateTransition())
         .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.COMMIT_PENDING, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
@@ -194,28 +202,31 @@ public abstract class TaskAttemptImpl im
         .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_TIMED_OUT, createFailRequestWhileRunningTransition())
         .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_FAIL_REQUEST, createFailRequestWhileRunningTransition())
         .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptEventType.TA_KILL_REQUEST, createKillRequestWhileRunningTransition())
-        .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.FAILED, TaskAttemptEventType.TA_TERMINATED, createContainerCompletedWhileRunningTransition())
+        .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptEventType.TA_NODE_FAILED, createNodeFailedWhileRunningTransition())
+        .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.RUNNING, TaskAttemptEventType.TA_CONTAINER_TERMINATING, createContainerTerminatingWhileRunningTransition())
+        .addTransition(TaskAttemptState.COMMIT_PENDING, TaskAttemptState.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, createContainerCompletedWhileRunningTransition())
 
-        .addTransition(TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptState.KILLED, TaskAttemptEventType.TA_TERMINATED, createTerminatedTransition())
+        .addTransition(TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptState.KILLED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, createTerminatedTransition())
         .addTransition(TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptState.KILL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST))
+        .addTransition(TaskAttemptState.KILL_IN_PROGRESS, TaskAttemptState.KILL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING))
         
-        .addTransition(TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptState.FAILED, TaskAttemptEventType.TA_TERMINATED, createTerminatedTransition())
+        .addTransition(TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptState.FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATED, createTerminatedTransition())
         .addTransition(TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptState.FAIL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST))
+        .addTransition(TaskAttemptState.FAIL_IN_PROGRESS, TaskAttemptState.FAIL_IN_PROGRESS, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING))
         
         .addTransition(TaskAttemptState.KILLED, TaskAttemptState.KILLED, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptState.KILLED, TaskAttemptState.KILLED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_TERMINATED))
+        .addTransition(TaskAttemptState.KILLED, TaskAttemptState.KILLED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATED))
 
         .addTransition(TaskAttemptState.FAILED, TaskAttemptState.FAILED, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
-        .addTransition(TaskAttemptState.FAILED, TaskAttemptState.FAILED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_TERMINATED))
+        .addTransition(TaskAttemptState.FAILED, TaskAttemptState.FAILED, EnumSet.of(TaskAttemptEventType.TA_STARTED_REMOTELY, TaskAttemptEventType.TA_STATUS_UPDATE, TaskAttemptEventType.TA_COMMIT_PENDING, TaskAttemptEventType.TA_DONE, TaskAttemptEventType.TA_FAILED, TaskAttemptEventType.TA_FAIL_REQUEST, TaskAttemptEventType.TA_KILL_REQUEST, TaskAttemptEventType.TA_NODE_FAILED, TaskAttemptEventType.TA_CONTAINER_TERMINATING, TaskAttemptEventType.TA_CONTAINER_TERMINATED))
         
         // TODO XXX: FailRequest / KillRequest at SUCCEEDED need to consider Map / Reduce task.
         .addTransition(TaskAttemptState.SUCCEEDED, TaskAttemptState.SUCCEEDED, TaskAttemptEventType.TA_DIAGNOSTICS_UPDATE, DIAGNOSTIC_INFORMATION_UPDATE_TRANSITION)
         .addTransition(TaskAttemptState.SUCCEEDED, TaskAttemptState.FAILED, TaskAttemptEventType.TA_FAIL_REQUEST, createFailRequestAfterSuccessTransition())
         .addTransition(TaskAttemptState.SUCCEEDED, TaskAttemptState.KILLED, TaskAttemptEventType.TA_KILL_REQUEST, createKillRequestAfterSuccessTransition())
+        .addTransition(TaskAttemptState.SUCCEEDED, TaskAttemptState.KILLED, TaskAttemptEventType.TA_NODE_FAILED, createNodeFailedAfterSuccessTransition())
         .addTransition(TaskAttemptState.SUCCEEDED, TaskAttemptState.FAILED, TaskAttemptEventType.TA_TOO_MANY_FETCH_FAILURES, createTooManyFetchFailuresTransition())
-        .addTransition(TaskAttemptState.SUCCEEDED, TaskAttemptState.SUCCEEDED, EnumSet.of(TaskAttemptEventType.TA_TERMINATED, TaskAttemptEventType.TA_TIMED_OUT))
+        .addTransition(TaskAttemptState.SUCCEEDED, TaskAttemptState.SUCCEEDED, EnumSet.of(TaskAttemptEventType.TA_CONTAINER_TERMINATED, TaskAttemptEventType.TA_TIMED_OUT, TaskAttemptEventType.TA_CONTAINER_TERMINATING))
         
         
         .installTopology();
@@ -839,6 +850,7 @@ public abstract class TaskAttemptImpl im
       ta.sendEvent(createJobCounterUpdateEventTATerminated(ta, false,
           TaskAttemptState.FAILED));
       if (ta.getLaunchTime() != 0) {
+        // TODO XXX: For cases like this, recovery goes for a toss, since the the attempt will not exist in the history file.
         ta.sendEvent(new JobHistoryEvent(ta.jobId,
             createTaskAttemptUnsuccessfulCompletionEvent(ta,
                 TaskAttemptState.FAILED)));
@@ -851,9 +863,6 @@ public abstract class TaskAttemptImpl im
       // Send out events to the Task - indicating TaskAttemptFailure.
       ta.sendEvent(new TaskTAttemptEvent(ta.attemptId,
           TaskEventType.T_ATTEMPT_FAILED));
-
-      // TODO Informing the scheduler is only required if the event came in
-      // after the scheduler was asked to launch the task. Likely in a subclass.
     }
   }
 
@@ -886,9 +895,6 @@ public abstract class TaskAttemptImpl im
       // Send out events to the Task - indicating TaskAttemptFailure.
       ta.sendEvent(new TaskTAttemptEvent(ta.attemptId,
           TaskEventType.T_ATTEMPT_KILLED));
-
-      // TODO Informing the scheduler is only required if the event came in
-      // after the scheduler was asked to launch the task. Likely in a subclass.
     }
   }
   
@@ -960,10 +966,10 @@ public abstract class TaskAttemptImpl im
   protected static class FailRequestBeforeRunning extends FailRequest {
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
-      // XXX Remove Comment: Takes care of finish time, history, TaskEvent.
       super.transition(ta, event);
       // Inform the scheduler
-      ta.sendEvent(new AMSchedulerTAStopRequestEvent(ta.attemptId, true));
+      ta.sendEvent(new AMSchedulerEventTAEnded(ta.attemptId,
+          TaskAttemptState.FAILED));
       // Decrement speculator container request.
       ta.maybeSendSpeculatorContainerRelease();
       
@@ -983,7 +989,8 @@ public abstract class TaskAttemptImpl im
       // XXX Remove Comment: Takes care of finish time, history, TaskEvent.
       super.transition(ta, event);
       // Inform the scheduler
-      ta.sendEvent(new AMSchedulerTAStopRequestEvent(ta.attemptId, false));
+      ta.sendEvent(new AMSchedulerEventTAEnded(ta.attemptId,
+          TaskAttemptState.KILLED));
       // Decrement speculator container request.
       ta.maybeSendSpeculatorContainerRelease();
       
@@ -991,21 +998,53 @@ public abstract class TaskAttemptImpl im
     }
   }
 
+  protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent>
+      createNodeFailedBeforeRunningTransition() {
+    return new NodeFailedBeforeRunning();
+  }
+
+  protected static class NodeFailedBeforeRunning extends
+      KillRequestBeforeRunning {
+    @Override
+    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      TaskAttemptEventNodeFailed nfEvent = (TaskAttemptEventNodeFailed) event;
+      ta.addDiagnosticInfo(nfEvent.getDiagnosticInfo());
+    }
+  }
+
+  protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent>
+      createContainerTerminatingBeforeRunningTransition() {
+    return new ContainerTerminatingBeforeRunning();
+  }
+
+  protected static class ContainerTerminatingBeforeRunning extends
+      FailRequestBeforeRunning {
+    @Override
+    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      super.transition(ta, event);
+      TaskAttemptEventContainerTerminating tEvent = 
+          (TaskAttemptEventContainerTerminating) event;
+      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
+    }
+  }
+
   protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> 
       createContainerCompletedBeforeRunningTransition() {
     return new ContainerCompletedBeforeRunning();
   }
 
-  protected static class ContainerCompletedBeforeRunning extends FailRequest {
+  protected static class ContainerCompletedBeforeRunning extends
+      FailRequestBeforeRunning {
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
-      // History, Inform Task, finishTime handled by FailRequest
-      // Decrement speculator container request.
-      ta.maybeSendSpeculatorContainerRelease();
-      
+      ta.sendTaskAttemptCleanupEvent();
+
+      TaskAttemptEventContainerTerminated tEvent =
+          (TaskAttemptEventContainerTerminated) event;
+      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
+
       // TODO XXX: Maybe other counters: Failed, Killed, etc.
-      // TODO XXX XXX: May need to inform the scheduler.
     }
   }
 
@@ -1073,7 +1112,8 @@ public abstract class TaskAttemptImpl im
       ta.sendEvent(createJobCounterUpdateEventSlotMillis(ta));
 
       // Inform the Scheduler.
-      ta.sendEvent(new AMSchedulerTASucceededEvent(ta.attemptId));
+      ta.sendEvent(new AMSchedulerEventTAEnded(ta.attemptId,
+          TaskAttemptState.SUCCEEDED));
       
       // Inform the task.
       ta.sendEvent(new TaskTAttemptEvent(ta.attemptId,
@@ -1118,19 +1158,50 @@ public abstract class TaskAttemptImpl im
       // TODO Speculator does not need to go out. maybeSend... will take care of this for now.
     }
   }
-  
+
+  protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent>
+      createNodeFailedWhileRunningTransition() {
+    return new NodeFailedWhileRunning();
+  }
+
+  protected static class NodeFailedWhileRunning extends FailRequestWhileRunning {
+    @Override
+    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      TaskAttemptEventNodeFailed nfEvent = (TaskAttemptEventNodeFailed) event;
+      ta.addDiagnosticInfo(nfEvent.getDiagnosticInfo());
+    }
+  }
+
+  protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent>
+      createContainerTerminatingWhileRunningTransition() {
+    return new ContainerTerminatingWhileRunning();
+  }
+
+  protected static class ContainerTerminatingWhileRunning extends
+      FailRequestWhileRunning {
+    @Override
+    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      super.transition(ta, event);
+      TaskAttemptEventContainerTerminating tEvent =
+          (TaskAttemptEventContainerTerminating) event;
+      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
+    }
+  }
+
   protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent>
       createContainerCompletedWhileRunningTransition() {
     return new ContaienrCompletedWhileRunning();
   }
 
   protected static class ContaienrCompletedWhileRunning extends
-      ContainerCompletedBeforeRunning {
+      FailRequestWhileRunning {
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       super.transition(ta, event);
       ta.sendTaskAttemptCleanupEvent();
-      ta.taskHeartbeatHandler.unregister(ta.attemptId);
+      TaskAttemptEventContainerTerminated tEvent =
+          (TaskAttemptEventContainerTerminated) event;
+      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
     }
   }
 
@@ -1145,6 +1216,9 @@ public abstract class TaskAttemptImpl im
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       ta.sendTaskAttemptCleanupEvent();
+      TaskAttemptEventContainerTerminated tEvent =
+          (TaskAttemptEventContainerTerminated) event;
+      ta.addDiagnosticInfo(tEvent.getDiagnosticInfo());
     }
 
   }
@@ -1184,6 +1258,19 @@ public abstract class TaskAttemptImpl im
       ta.sendTaskAttemptCleanupEvent();
     }
   }
+  
+  protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent>
+      createNodeFailedAfterSuccessTransition() {
+    return new NodeFailedAfterSuccess();
+  }
+  
+  protected static class NodeFailedAfterSuccess extends KillRequestAfterSuccess {
+    @Override
+    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      TaskAttemptEventNodeFailed nfEvent = (TaskAttemptEventNodeFailed) event;
+      ta.addDiagnosticInfo(nfEvent.getDiagnosticInfo());
+    }
+  }
 
   protected SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> 
       createTooManyFetchFailuresTransition() {

Modified: 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/impl/TaskImpl.java
URL: http://svn.apache.org/viewvc/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/impl/TaskImpl.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- 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/impl/TaskImpl.java (original)
+++ 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/impl/TaskImpl.java Mon Oct 15 21:09:59 2012
@@ -44,7 +44,6 @@ import org.apache.hadoop.mapreduce.jobhi
 import org.apache.hadoop.mapreduce.jobhistory.TaskFailedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskFinishedEvent;
 import org.apache.hadoop.mapreduce.jobhistory.TaskStartedEvent;
-import org.apache.hadoop.security.ssl.SSLFactory;
 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.TaskAttemptCompletionEvent;
@@ -73,7 +72,6 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskEventType;
 import org.apache.hadoop.mapreduce.v2.app2.job.event.TaskTAttemptEvent;
 import org.apache.hadoop.mapreduce.v2.app2.metrics.MRAppMetrics;
-import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTAStopRequestEvent;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.security.Credentials;
 import org.apache.hadoop.security.token.Token;
@@ -859,15 +857,9 @@ public abstract class TaskImpl implement
       if (attempt.getAssignedContainerMgrAddress() != null) {
         //container was assigned
         // TOOD XXX: What else changes other than this one transition.
-        
         // This can originate from TOO_MANY_FETCH_FAILURES -> the Container may still be running. Ask the scheduler to KILL it.
-        // TODO XXX: Send out a TA_STOP_REQUEST. or the Task sends this out directly, considering the TaskAttempt may already have completed.
-//        task.eventHandler.handle(new ContainerFailedEvent(attempt.getID(), 
-//            attempt.getAssignedContainerMgrAddress()));
-        // TODO XXX: This is not required here. TaskAttempt should be sending out the STOP_REQUEST
-        task.eventHandler.handle(new AMSchedulerTAStopRequestEvent(castEvent.getTaskAttemptID(), true));
       }
-      
+
       task.finishedAttempts++;
       if (task.failedAttempts < task.maxAttempts) {
         task.handleTaskAttemptCompletion(

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/launcher/ContainerLauncherImpl.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/launcher/ContainerLauncherImpl.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/launcher/ContainerLauncherImpl.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/launcher/ContainerLauncherImpl.java Mon Oct 15 21:09:59 2012
@@ -38,9 +38,11 @@ import org.apache.hadoop.mapreduce.MRJob
 import org.apache.hadoop.mapreduce.v2.app2.AppContext;
 import org.apache.hadoop.mapreduce.v2.app2.rm.NMCommunicatorEvent;
 import org.apache.hadoop.mapreduce.v2.app2.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerEvent;
 import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerEventLaunchFailed;
 import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerEventLaunched;
 import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerEventStopFailed;
+import org.apache.hadoop.mapreduce.v2.app2.rm.container.AMContainerEventType;
 import org.apache.hadoop.net.NetUtils;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.security.token.Token;
@@ -65,7 +67,6 @@ import com.google.common.util.concurrent
 // TODO XXX: See what part of this lifecycle and state management can be simplified.
 // Ideally, no state - only sendStart / sendStop.
 
-// TODO XXX: ShufflePort needs to make it over to the TaskAttempt.
 // TODO XXX: Review this entire code and clean it up.
 
 /**
@@ -210,14 +211,18 @@ public class ContainerLauncherImpl exten
               .newRecord(StopContainerRequest.class);
             stopRequest.setContainerId(this.containerID);
             proxy.stopContainer(stopRequest);
-
+            // If stopContainer returns without an error, assuming the stop made
+            // it over to the NodeManager.
+          context.getEventHandler().handle(
+              new AMContainerEvent(containerID, AMContainerEventType.C_NM_STOP_SENT));
         } catch (Throwable t) {
 
           // ignore the cleanup failure
           String message = "cleanup failed for container "
             + this.containerID + " : "
             + StringUtils.stringifyException(t);
-          context.getEventHandler().handle(new AMContainerEventStopFailed(containerID, message));
+          context.getEventHandler().handle(
+              new AMContainerEventStopFailed(containerID, message));
           LOG.warn(message);
           this.state = ContainerState.DONE;
           return;
@@ -228,11 +233,6 @@ public class ContainerLauncherImpl exten
         }
         this.state = ContainerState.DONE;
       }
-      // TODO XXX: NO STOPPED event. Waiting for the RM to get back.
-      // after killing, send killed event to task attempt
-//      context.getEventHandler().handle(
-//          new TaskAttemptEvent(this.taskAttemptID,
-//              TaskAttemptEventType.TA_CONTAINER_CLEANED));
     }
   }
 

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/recover/RecoveryService.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/recover/RecoveryService.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/recover/RecoveryService.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/recover/RecoveryService.java Mon Oct 15 21:09:59 2012
@@ -66,8 +66,8 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerEventContainerCompleted;
 import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerEventType;
 import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTALaunchRequestEvent;
-import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTAStopRequestEvent;
-import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerTASucceededEvent;
+import org.apache.hadoop.mapreduce.v2.app2.rm.AMSchedulerEventTAEnded;
+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.NMCommunicatorLaunchRequestEvent;
 import org.apache.hadoop.mapreduce.v2.app2.rm.RMCommunicatorContainerDeAllocateRequestEvent;
@@ -85,6 +85,7 @@ import org.apache.hadoop.mapreduce.v2.ap
 import org.apache.hadoop.mapreduce.v2.jobhistory.JobHistoryUtils;
 import org.apache.hadoop.mapreduce.v2.util.MRBuilderUtils;
 import org.apache.hadoop.yarn.Clock;
+import org.apache.hadoop.yarn.YarnException;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
@@ -468,32 +469,40 @@ public class RecoveryService extends Com
       } 
       
       // Handle Events which may be sent to the scheduler.
-      else if (event.getType() == AMSchedulerEventType.S_TA_SUCCEEDED) {
-        // Inform the container that the task attempt succeeded.
-        AMSchedulerTASucceededEvent sEvent = (AMSchedulerTASucceededEvent)event;
-        
-        // Leaving the event in the map - for TA failure after success.
-        ContainerId containerId = attemptToContainerMap.get(sEvent.getAttemptID());
-        actualHandler.handle(new AMContainerTASucceededEvent(containerId,
-            sEvent.getAttemptID()));
-        return;
-        // XXX (Post-3902)tal.unregister happens here. Ensure THH handles it
-        // correctly in case of recovery.
-      }
-      else if (event.getType() == AMSchedulerEventType.S_TA_STOP_REQUEST) {
+      else if (event.getType() == AMSchedulerEventType.S_TA_ENDED) {
         // Tell the container to stop.
-        AMSchedulerTAStopRequestEvent sEvent = (AMSchedulerTAStopRequestEvent) event;
-        ContainerId containerId = attemptToContainerMap.get(sEvent.getAttemptID());
-        actualHandler.handle(new AMContainerEvent(containerId,
-            AMContainerEventType.C_STOP_REQUEST));
-        return;
-        // XXX (Post-3902)chh.unregister happens here. Ensure THH handles it
-        // correctly in case of recovery.
+        AMSchedulerEventTAEnded sEvent = (AMSchedulerEventTAEnded) event;
+        ContainerId containerId = attemptToContainerMap.get(sEvent
+            .getAttemptID());
+        switch (sEvent.getState()) {
+        case FAILED: 
+        case KILLED:
+          actualHandler.handle(new AMContainerEvent(containerId,
+              AMContainerEventType.C_STOP_REQUEST));
+          return;
+          // XXX (Post-3902)chh.unregister happens here. Ensure THH handles it
+          // correctly in case of recovery.
+        case SUCCEEDED:
+          // Inform the container that the task attempt succeeded.
+          // Leaving the event in the map - for TA failure after success.
+          actualHandler.handle(new AMContainerTASucceededEvent(containerId,
+              sEvent.getAttemptID()));
+          return;
+          // XXX (Post-3902)tal.unregister happens here. Ensure THH handles it
+          // correctly in case of recovery.
+        default:
+            throw new YarnException("Invalid state " + sEvent.getState());
+        }
       }
       
-      // Ignore de-allocate requests for the container.
+      // De-allocate containers used by previous attempts immediately.
       else if (event.getType() == NMCommunicatorEventType.CONTAINER_STOP_REQUEST) {
         // Ignore. Unless we start relying on a successful NM.stopContainer() call.
+        NMCommunicatorEvent nEvent = (NMCommunicatorEvent)event;
+        ContainerId cId = nEvent.getContainerId();
+        ContainerStatus cs = BuilderUtils.newContainerStatus(cId,
+            ContainerState.COMPLETE, "", 0);
+        actualHandler.handle(new AMContainerEventCompleted(cs));
         return;
       }
       

Added: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventTAEnded.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventTAEnded.java?rev=1398523&view=auto
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventTAEnded.java (added)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventTAEnded.java Mon Oct 15 21:09:59 2012
@@ -0,0 +1,41 @@
+/**
+ * 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;
+
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptId;
+import org.apache.hadoop.mapreduce.v2.api.records.TaskAttemptState;
+
+public class AMSchedulerEventTAEnded extends AMSchedulerEvent {
+
+  private final TaskAttemptId attemptId;
+  private TaskAttemptState state;
+
+  public AMSchedulerEventTAEnded(TaskAttemptId attemptId, TaskAttemptState state) {
+    super(AMSchedulerEventType.S_TA_ENDED);
+    this.attemptId = attemptId;
+    this.state = state;
+  }
+
+  public TaskAttemptId getAttemptID() {
+    return this.attemptId;
+  }
+
+  public TaskAttemptState getState() {
+    return this.state;
+  }
+}
\ No newline at end of file

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventType.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventType.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventType.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/AMSchedulerEventType.java Mon Oct 15 21:09:59 2012
@@ -3,9 +3,7 @@ package org.apache.hadoop.mapreduce.v2.a
 public enum AMSchedulerEventType {
   //Producer: TaskAttempt
   S_TA_LAUNCH_REQUEST,
-  S_TA_STOP_REQUEST, // Maybe renamed to S_TA_END / S_TA_ABNORMAL_END
-  S_TA_SUCCEEDED,
-  S_TA_ENDED,
+  S_TA_ENDED, // Annotated with FAILED/KILLED/SUCCEEDED.
 
   //Producer: RMCommunicator
   S_CONTAINERS_ALLOCATED,

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/RMContainerAllocator.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/RMContainerAllocator.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/RMContainerAllocator.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/RMContainerAllocator.java Mon Oct 15 21:09:59 2012
@@ -48,6 +48,7 @@ import org.apache.hadoop.mapreduce.jobhi
 import org.apache.hadoop.mapreduce.jobhistory.NormalizedResourceEvent;
 import org.apache.hadoop.mapreduce.v2.api.records.JobId;
 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.TaskType;
 import org.apache.hadoop.mapreduce.v2.app2.AppContext;
 import org.apache.hadoop.mapreduce.v2.app2.job.Job;
@@ -332,18 +333,20 @@ public class RMContainerAllocator extend
       recalculateReduceSchedule = true;
       handleTaLaunchRequest((AMSchedulerTALaunchRequestEvent) sEvent);
       break;
-    case S_TA_STOP_REQUEST: // Effectively means a failure.
+    case S_TA_ENDED: // Effectively means a failure.
       recalculateReduceSchedule = true;
-      handleTaStopRequest((AMSchedulerTAStopRequestEvent) sEvent);
-      break;
-    case S_TA_SUCCEEDED:
-      recalculateReduceSchedule = true;
-      handleTaSucceededRequest((AMSchedulerTASucceededEvent) sEvent);
-      break;
-    case S_TA_ENDED:
-      recalculateReduceSchedule = true;
-      // TODO XXX XXX: Not generated yet. Depends on E05 etc. Also look at
-      // TaskAttempt transitions.
+      AMSchedulerEventTAEnded event = (AMSchedulerEventTAEnded)sEvent;
+      switch(event.getState()) {
+      case FAILED:
+      case KILLED:
+        handleTaStopRequest((AMSchedulerEventTAEnded) sEvent);
+        break;
+      case SUCCEEDED:
+        handleTaSucceededRequest(event);
+        break;
+      default:
+        throw new YarnException("Unexecpted TA_ENDED state: " + event.getState()); 
+      }
       break;
     case S_CONTAINERS_ALLOCATED:
       // Conditional recalculateReduceSchedule
@@ -391,7 +394,7 @@ public class RMContainerAllocator extend
     }
   }
 
-  private void handleTaStopRequest(AMSchedulerTAStopRequestEvent event) {
+  private void handleTaStopRequest(AMSchedulerEventTAEnded event) {
     TaskAttemptId aId = event.getAttemptID();
     attemptToLaunchRequestMap.remove(aId);
     // TODO XXX: This remove may need to be deferred. Possible for a SUCCESSFUL taskAttempt to fail,
@@ -410,7 +413,7 @@ public class RMContainerAllocator extend
           // stopped.
           sendEvent(new AMNodeEventTaskAttemptEnded(containerMap
               .get(containerId).getContainer().getNodeId(), containerId,
-              event.getAttemptID(), event.failed()));
+              event.getAttemptID(), event.getState() == TaskAttemptState.FAILED));
         } else {
           LOG.warn("Received a STOP request for absent taskAttempt: "
               + event.getAttemptID());
@@ -422,7 +425,7 @@ public class RMContainerAllocator extend
     }
   }
   
-  private void handleTaSucceededRequest(AMSchedulerTASucceededEvent event) {
+  private void handleTaSucceededRequest(AMSchedulerEventTAEnded event) {
     // TODO XXX Remember the assigned containerId even after task success.
     // Required for TOO_MANY_FETCH_FAILURES
     attemptToLaunchRequestMap.remove(event.getAttemptID());

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEvent.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEvent.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEvent.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEvent.java Mon Oct 15 21:09:59 2012
@@ -1,13 +1,27 @@
+/**
+ * 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 org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.hadoop.yarn.event.AbstractEvent;
 
-// TODO: Implement.
-
 public class AMContainerEvent extends AbstractEvent<AMContainerEventType> {
 
-
   private final ContainerId containerId;
   
   public AMContainerEvent(ContainerId containerId, AMContainerEventType type) {

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventStopFailed.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventStopFailed.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventStopFailed.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventStopFailed.java Mon Oct 15 21:09:59 2012
@@ -10,7 +10,7 @@ public class AMContainerEventStopFailed 
   private final String message;
 
   public AMContainerEventStopFailed(ContainerId containerId, String message) {
-    super(containerId, AMContainerEventType.C_STOP_FAILED);
+    super(containerId, AMContainerEventType.C_NM_STOP_FAILED);
     this.message = message;
   }
 

Modified: hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventType.java
URL: http://svn.apache.org/viewvc/hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventType.java?rev=1398523&r1=1398522&r2=1398523&view=diff
==============================================================================
--- hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventType.java (original)
+++ hadoop/common/branches/MR-3902/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app2/src/main/java/org/apache/hadoop/mapreduce/v2/app2/rm/container/AMContainerEventType.java Mon Oct 15 21:09:59 2012
@@ -1,34 +1,53 @@
+/**
+* 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;
 
 public enum AMContainerEventType {
 
-  // TODO Merge START/LAUNCH, STOP/HALT
-  
   //Producer: Scheduler
-  C_START_REQUEST,
+  C_LAUNCH_REQUEST,
   C_ASSIGN_TA,
   
   //Producer: NMCommunicator
   C_LAUNCHED,
   C_LAUNCH_FAILED, // TODO XXX: Send a diagnostic update message to the TaskAttempts assigned to this container ?
-  
+
   //Producer: TAL: PULL_TA is a sync call.
   C_PULL_TA,
-  
+
   //Producer: Scheduler via TA
-  C_TA_SUCCEEDED,
-  
-  //Producer:RMCommunicator
+  C_TA_SUCCEEDED, // maybe change this to C_TA_FINISHED with a status.
+
+  //Producer: RMCommunicator
   C_COMPLETED,
+  
+  //Producer: RMCommunicator, AMNode
   C_NODE_FAILED,
   
   //Producer: TA-> Scheduler -> Container (in case of failure etc)
   //          Scheduler -> Container (in case of pre-emption etc)
-  //          Node -> Container (in case of Node unhealthy etc)
+  //          Node -> Container (in case of Node blacklisted etc)
   C_STOP_REQUEST,
   
   //Producer: NMCommunicator
-  C_STOP_FAILED,
+  C_NM_STOP_FAILED,
+  C_NM_STOP_SENT,
   
   //Producer: ContainerHeartbeatHandler
   C_TIMED_OUT,



Mime
View raw message