You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by ss...@apache.org on 2014/03/15 01:19:01 UTC

git commit: TEZ-936. Remove unused event types from AMSchedulerEventType. (sseth)

Repository: incubator-tez
Updated Branches:
  refs/heads/master ed8c288a6 -> 98d5a31cb


TEZ-936. Remove unused event types from AMSchedulerEventType. (sseth)


Project: http://git-wip-us.apache.org/repos/asf/incubator-tez/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-tez/commit/98d5a31c
Tree: http://git-wip-us.apache.org/repos/asf/incubator-tez/tree/98d5a31c
Diff: http://git-wip-us.apache.org/repos/asf/incubator-tez/diff/98d5a31c

Branch: refs/heads/master
Commit: 98d5a31cb7f2963a7395c69f4942543dc050d82d
Parents: ed8c288
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Mar 14 17:18:39 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Mar 14 17:18:39 2014 -0700

----------------------------------------------------------------------
 .../rm/AMSchedulerEventContainerCompleted.java  | 37 ----------
 .../rm/AMSchedulerEventContainersAllocated.java | 48 -------------
 .../tez/dag/app/rm/AMSchedulerEventType.java    |  6 --
 .../dag/app/rm/TaskSchedulerEventHandler.java   |  4 --
 .../dag/app/rm/container/AMContainerImpl.java   | 14 ----
 .../dag/app/rm/container/TestAMContainer.java   | 75 +++++++-------------
 6 files changed, 25 insertions(+), 159 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/98d5a31c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainerCompleted.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainerCompleted.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainerCompleted.java
deleted file mode 100644
index 5dd36b0..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainerCompleted.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/**
- * 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.tez.dag.app.rm;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-
-public class AMSchedulerEventContainerCompleted extends AMSchedulerEvent {
-
-  private final ContainerId containerId;
-  
-  public AMSchedulerEventContainerCompleted(ContainerId containerId) {
-    super(AMSchedulerEventType.S_CONTAINER_COMPLETED);;
-    this.containerId = containerId;
-  }
-  
-  public ContainerId getContainerId() {
-    return this.containerId;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/98d5a31c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainersAllocated.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainersAllocated.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainersAllocated.java
deleted file mode 100644
index eae67d9..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventContainersAllocated.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
-* 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.tez.dag.app.rm;
-
-import java.util.List;
-
-import org.apache.hadoop.yarn.api.records.ContainerId;
-
-public class AMSchedulerEventContainersAllocated extends AMSchedulerEvent {
-
-  private final List<ContainerId> containerIds;
-  private final boolean headRoomChanged;
-
-  // TODO Maybe distinguish between newly allocated containers and 
-  // existing containers being re-used.
-  // headRoomChanged is a strange API - making an assumption about how the
-  // scheduler will use this info.
-  public AMSchedulerEventContainersAllocated(List<ContainerId> containerIds,
-      boolean headRoomChanged) {
-    super(AMSchedulerEventType.S_CONTAINERS_ALLOCATED);
-    this.containerIds = containerIds;
-    this.headRoomChanged = headRoomChanged;
-  }
-
-  public List<ContainerId> getContainerIds() {
-    return this.containerIds;
-  }
-
-  public boolean didHeadroomChange() {
-    return headRoomChanged;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/98d5a31c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventType.java
index 8a4c371..053146d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/AMSchedulerEventType.java
@@ -23,12 +23,6 @@ public enum AMSchedulerEventType {
   S_TA_LAUNCH_REQUEST,
   S_TA_ENDED, // Annotated with FAILED/KILLED/SUCCEEDED.
 
-  //Producer: RMCommunicator
-  S_CONTAINERS_ALLOCATED,
-
-  //Producer: Container. (Maybe RMCommunicator)
-  S_CONTAINER_COMPLETED,
-
   //Producer: Node
   S_NODE_BLACKLISTED,
   S_NODE_UNBLACKLISTED,

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/98d5a31c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
index 66ef819..69f2afe 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/TaskSchedulerEventHandler.java
@@ -142,10 +142,6 @@ public class TaskSchedulerEventHandler extends AbstractService
     case S_CONTAINER_DEALLOCATE:
       handleContainerDeallocate((AMSchedulerEventDeallocateContainer)sEvent);
       break;
-    case S_CONTAINERS_ALLOCATED:
-      break;
-    case S_CONTAINER_COMPLETED:
-      break;
     case S_NODE_UNBLACKLISTED:
       // fall through
     case S_NODE_BLACKLISTED:

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/98d5a31c/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
index d782be2..8fb090f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/rm/container/AMContainerImpl.java
@@ -51,7 +51,6 @@ import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerPreempted;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminated;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminating;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventNodeFailed;
-import org.apache.tez.dag.app.rm.AMSchedulerEventContainerCompleted;
 import org.apache.tez.dag.app.rm.AMSchedulerEventDeallocateContainer;
 import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
 import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
@@ -530,7 +529,6 @@ public class AMContainerImpl implements AMContainer {
       container.sendTerminatedToTaskAttempt(event.getTaskAttemptId(),
           "AMScheduler Error: TaskAttempt allocated to unlaunched container: " +
               container.getContainerId());
-      container.sendCompletedToScheduler();
       container.deAllocate();
       LOG.warn("Unexpected TA Assignment: TAId: " + event.getTaskAttemptId() +
           "  for ContainerId: " + container.getContainerId() +
@@ -543,7 +541,6 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       AMContainerEventCompleted event = (AMContainerEventCompleted)cEvent;
-      container.sendCompletedToScheduler();
       String diag = event.getContainerStatus().getDiagnostics();
       if (!(diag == null || diag.equals(""))) {
         LOG.info("Container " + container.getContainerId()
@@ -556,8 +553,6 @@ public class AMContainerImpl implements AMContainer {
       SingleArcTransition<AMContainerImpl, AMContainerEvent> {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
-      // TODO why are these sent. no need to send these now.
-      container.sendCompletedToScheduler();
       container.deAllocate();
     }
   }
@@ -567,7 +562,6 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.sendCompletedToScheduler();
       container.deAllocate();
     }
   }
@@ -576,7 +570,6 @@ public class AMContainerImpl implements AMContainer {
     @Override
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.sendCompletedToScheduler();
       container.deAllocate();
       LOG.info(
           "Unexpected event type: " + cEvent.getType() + " while in state: " +
@@ -675,7 +668,6 @@ public class AMContainerImpl implements AMContainer {
       container.containerLocalResources = null;
       container.additionalLocalResources = null;
       container.unregisterFromTAListener();
-      container.sendCompletedToScheduler();
       String diag = event.getContainerStatus().getDiagnostics();
       if (!(diag == null || diag.equals(""))) {
         LOG.info("Container " + container.getContainerId()
@@ -1018,7 +1010,6 @@ public class AMContainerImpl implements AMContainer {
       }
       container.containerLocalResources = null;
       container.additionalLocalResources = null;
-      container.sendCompletedToScheduler();
     }
   }
 
@@ -1049,7 +1040,6 @@ public class AMContainerImpl implements AMContainer {
       extends ErrorBaseTransition {
     public void transition(AMContainerImpl container, AMContainerEvent cEvent) {
       super.transition(container, cEvent);
-      container.sendCompletedToScheduler();
     }
   }
 
@@ -1110,10 +1100,6 @@ public class AMContainerImpl implements AMContainer {
     sendEvent(new AMSchedulerEventDeallocateContainer(containerId));
   }
 
-  protected void sendCompletedToScheduler() {
-    sendEvent(new AMSchedulerEventContainerCompleted(containerId));
-  }
-
   protected void sendTerminatedToTaskAttempt(
       TezTaskAttemptID taId, String message) {
     sendEvent(new TaskAttemptEventContainerTerminated(taId, message));

http://git-wip-us.apache.org/repos/asf/incubator-tez/blob/98d5a31c/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
index b78704a..6bd8fcc 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/rm/container/TestAMContainer.java
@@ -136,8 +136,7 @@ public class TestAMContainer {
 
     wc.containerCompleted(false);
     wc.verifyState(AMContainerState.COMPLETED);
-    // 1 Scheduler completed event.
-    wc.verifyCountAndGetOutgoingEvents(1);
+    wc.verifyNoOutgoingEvents();
     verify(wc.tal).unregisterRunningContainer(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -192,8 +191,7 @@ public class TestAMContainer {
 
     wc.containerCompleted(false);
     wc.verifyState(AMContainerState.COMPLETED);
-    // 1 Scheduler completed event.
-    wc.verifyCountAndGetOutgoingEvents(1);
+    wc.verifyNoOutgoingEvents();
     verify(wc.tal).unregisterRunningContainer(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -226,8 +224,7 @@ public class TestAMContainer {
 
     wc.containerCompleted(false);
     wc.verifyState(AMContainerState.COMPLETED);
-    // 1 Scheduler completed event.
-    wc.verifyCountAndGetOutgoingEvents(1);
+    wc.verifyNoOutgoingEvents();
     verify(wc.tal).unregisterRunningContainer(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -265,8 +262,7 @@ public class TestAMContainer {
 
     wc.containerCompleted(false);
     wc.verifyState(AMContainerState.COMPLETED);
-    // 1 Scheduler completed event.
-    wc.verifyCountAndGetOutgoingEvents(1);
+    wc.verifyNoOutgoingEvents();
     verify(wc.tal).unregisterRunningContainer(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
@@ -304,11 +300,10 @@ public class TestAMContainer {
     wc.nmStopSent();
     wc.containerCompleted(false);
     // 1 Inform scheduler. 2 TERMINATED to TaskAttempt.
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertNull(wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -344,11 +339,10 @@ public class TestAMContainer {
     wc.nmStopSent();
     wc.containerCompleted(false);
     // 1 Inform scheduler. 2 TERMINATED to TaskAttempt.
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertNull(wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -383,11 +377,10 @@ public class TestAMContainer {
     wc.nmStopSent();
     wc.containerCompleted(false);
     // 1 Inform scheduler. 2 TERMINATED to TaskAttempt.
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(3);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertNull(wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -418,10 +411,9 @@ public class TestAMContainer {
     // TODO Should this be an RM DE-ALLOCATE instead ?
 
     wc.containerCompleted(false);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertFalse(wc.amContainer.isInErrorState());
 
@@ -450,27 +442,22 @@ public class TestAMContainer {
         AMSchedulerEventType.S_CONTAINER_DEALLOCATE);
 
     wc.containerCompleted(false);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     // Valid transition. Container complete, but not with an error.
     assertFalse(wc.amContainer.isInErrorState());
   }
 
-  @SuppressWarnings("rawtypes")
   @Test
   public void testContainerCompletedAtAllocated() {
     WrappedContainer wc = new WrappedContainer();
-    List<Event> outgoingEvents;
     wc.verifyState(AMContainerState.ALLOCATED);
 
     wc.containerCompleted(false);
     wc.verifyState(AMContainerState.COMPLETED);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
-    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+    wc.verifyNoOutgoingEvents();
 
     assertFalse(wc.amContainer.isInErrorState());
   }
@@ -493,9 +480,8 @@ public class TestAMContainer {
     verify(wc.tal).registerRunningContainer(wc.containerID);
     verify(wc.tal).unregisterRunningContainer(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertFalse(wc.amContainer.isInErrorState());
@@ -526,9 +512,8 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertFalse(wc.amContainer.isInErrorState());
@@ -562,9 +547,8 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertFalse(wc.amContainer.isInErrorState());
@@ -598,9 +582,8 @@ public class TestAMContainer {
     verify(wc.chh).register(wc.containerID);
     verify(wc.chh).unregister(wc.containerID);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED,
         TaskAttemptEventType.TA_CONTAINER_PREEMPTED);
 
     assertFalse(wc.amContainer.isInErrorState());
@@ -685,10 +668,9 @@ public class TestAMContainer {
     }
 
     wc.containerCompleted(false);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertFalse(wc.amContainer.isInErrorState());
   }
@@ -731,9 +713,7 @@ public class TestAMContainer {
     assertFalse(wc.amContainer.isInErrorState());
 
     wc.containerCompleted(false);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
-    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+    wc.verifyNoOutgoingEvents();
 
     assertNull(wc.amContainer.getRunningTaskAttempt());
     assertEquals(0, wc.amContainer.getQueuedTaskAttempts().size());
@@ -775,10 +755,9 @@ public class TestAMContainer {
     }
 
     wc.containerCompleted(false);
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(2);
+    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
     verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        TaskAttemptEventType.TA_CONTAINER_TERMINATED,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+        TaskAttemptEventType.TA_CONTAINER_TERMINATED);
 
     assertFalse(wc.amContainer.isInErrorState());
     assertNull(wc.amContainer.getRunningTaskAttempt());
@@ -818,11 +797,9 @@ public class TestAMContainer {
     assertEquals(2, wc.amContainer.getAllTaskAttempts().size());
   }
 
-  @SuppressWarnings("rawtypes")
   @Test
   public void testDuplicateCompletedEvents() {
     WrappedContainer wc = new WrappedContainer();
-    List<Event> outgoingEvents;
 
     wc.launchContainer();
     wc.containerLaunched();
@@ -839,9 +816,7 @@ public class TestAMContainer {
     wc.containerCompleted(false);
     wc.verifyState(AMContainerState.COMPLETED);
 
-    outgoingEvents = wc.verifyCountAndGetOutgoingEvents(1);
-    verifyUnOrderedOutgoingEventTypes(outgoingEvents,
-        AMSchedulerEventType.S_CONTAINER_COMPLETED);
+    wc.verifyNoOutgoingEvents();
 
     wc.containerCompleted(false);
     wc.verifyNoOutgoingEvents();