You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by zj...@apache.org on 2015/11/25 15:02:21 UTC

[1/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Repository: tez
Updated Branches:
  refs/heads/master c4487f966 -> 28f30b0ef


http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
index 0d6cbcb..7082ca7 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/main/java/org/apache/tez/dag/history/logging/ats/HistoryEventTimelineConversion.java
@@ -50,7 +50,7 @@ import org.apache.tez.dag.history.events.TaskFinishedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.history.events.VertexFinishedEvent;
 import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
 import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.history.logging.EntityTypes;
 import org.apache.tez.dag.history.utils.DAGUtils;
@@ -113,15 +113,14 @@ public class HistoryEventTimelineConversion {
       case TASK_ATTEMPT_FINISHED:
         timelineEntity = convertTaskAttemptFinishedEvent((TaskAttemptFinishedEvent) historyEvent);
         break;
-      case VERTEX_PARALLELISM_UPDATED:
-        timelineEntity = convertVertexParallelismUpdatedEvent(
-            (VertexParallelismUpdatedEvent) historyEvent);
+      case VERTEX_CONFIGURE_DONE:
+        timelineEntity = convertVertexReconfigureDoneEvent(
+            (VertexConfigurationDoneEvent) historyEvent);
         break;
       case DAG_RECOVERED:
         timelineEntity = convertDAGRecoveredEvent(
             (DAGRecoveredEvent) historyEvent);
         break;
-      case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
       case VERTEX_COMMIT_STARTED:
       case VERTEX_GROUP_COMMIT_STARTED:
       case VERTEX_GROUP_COMMIT_FINISHED:
@@ -657,8 +656,8 @@ public class HistoryEventTimelineConversion {
     return atsEntity;
   }
 
-  private static TimelineEntity convertVertexParallelismUpdatedEvent(
-      VertexParallelismUpdatedEvent event) {
+  private static TimelineEntity convertVertexReconfigureDoneEvent(
+      VertexConfigurationDoneEvent event) {
     TimelineEntity atsEntity = new TimelineEntity();
     atsEntity.setEntityId(event.getVertexID().toString());
     atsEntity.setEntityType(EntityTypes.TEZ_VERTEX_ID.name());
@@ -669,8 +668,8 @@ public class HistoryEventTimelineConversion {
         event.getVertexID().getDAGId().toString());
 
     TimelineEvent updateEvt = new TimelineEvent();
-    updateEvt.setEventType(HistoryEventType.VERTEX_PARALLELISM_UPDATED.name());
-    updateEvt.setTimestamp(event.getUpdateTime());
+    updateEvt.setEventType(HistoryEventType.VERTEX_CONFIGURE_DONE.name());
+    updateEvt.setTimestamp(event.getReconfigureDoneTime());
 
     Map<String,Object> eventInfo = new HashMap<String, Object>();
     if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) {
@@ -683,7 +682,6 @@ public class HistoryEventTimelineConversion {
       eventInfo.put(ATSConstants.UPDATED_EDGE_MANAGERS, updatedEdgeManagers);
     }
     eventInfo.put(ATSConstants.NUM_TASKS, event.getNumTasks());
-    eventInfo.put(ATSConstants.OLD_NUM_TASKS, event.getOldNumTasks());
     updateEvt.setEventInfo(eventInfo);
     atsEntity.addEvent(updateEvt);
 

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
----------------------------------------------------------------------
diff --git a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
index 8e589d2..7792c62 100644
--- a/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
+++ b/tez-plugins/tez-yarn-timeline-history/src/test/java/org/apache/tez/dag/history/logging/ats/TestHistoryEventTimelineConversion.java
@@ -70,12 +70,11 @@ import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
 import org.apache.tez.dag.history.events.TaskFinishedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
 import org.apache.tez.dag.history.events.VertexFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
 import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
 import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.history.logging.EntityTypes;
 import org.apache.tez.dag.history.utils.DAGUtils;
@@ -158,13 +157,13 @@ public class TestHistoryEventTimelineConversion {
           break;
         case VERTEX_INITIALIZED:
           event = new VertexInitializedEvent(tezVertexID, "v1", random.nextInt(), random.nextInt(),
-              random.nextInt(), "proc", null);
+              random.nextInt(), "proc", null, null);
           break;
         case VERTEX_STARTED:
           event = new VertexStartedEvent(tezVertexID, random.nextInt(), random.nextInt());
           break;
-        case VERTEX_PARALLELISM_UPDATED:
-          event = new VertexParallelismUpdatedEvent(tezVertexID, 1, null, null, null, 1);
+        case VERTEX_CONFIGURE_DONE:
+          event = new VertexConfigurationDoneEvent(tezVertexID, 0L, 1, null, null, null, true);
           break;
         case VERTEX_FINISHED:
           event = new VertexFinishedEvent(tezVertexID, "v1", 1, random.nextInt(), random.nextInt(),
@@ -184,7 +183,8 @@ public class TestHistoryEventTimelineConversion {
           break;
         case TASK_ATTEMPT_FINISHED:
           event = new TaskAttemptFinishedEvent(tezTaskAttemptID, "v1", random.nextInt(),
-              random.nextInt(), TaskAttemptState.FAILED, TaskAttemptTerminationCause.OUTPUT_LOST, null, null, null, 0, null, 0);
+              random.nextInt(), TaskAttemptState.FAILED, TaskAttemptTerminationCause.OUTPUT_LOST,
+              null, null, null, null, 0, null, 0);
           break;
         case CONTAINER_LAUNCHED:
           event = new ContainerLaunchedEvent(containerId, random.nextInt(),
@@ -193,9 +193,6 @@ public class TestHistoryEventTimelineConversion {
         case CONTAINER_STOPPED:
           event = new ContainerStoppedEvent(containerId, random.nextInt(), -1, applicationAttemptId);
           break;
-        case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-          event = new VertexRecoverableEventsGeneratedEvent();
-          break;
         case DAG_COMMIT_STARTED:
           event = new DAGCommitStartedEvent();
           break;
@@ -524,7 +521,7 @@ public class TestHistoryEventTimelineConversion {
     events.add(new DataEventDependencyInfo(lastDataEventTime, tezTaskAttemptID));
 
     TaskAttemptFinishedEvent event = new TaskAttemptFinishedEvent(tezTaskAttemptID, vertexName,
-        startTime, finishTime, state, error, diagnostics, counters, events, creationTime,
+        startTime, finishTime, state, error, diagnostics, counters, events, null, creationTime,
         tezTaskAttemptID, allocationTime);
     TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
     Assert.assertEquals(tezTaskAttemptID.toString(), timelineEntity.getEntityId());
@@ -668,7 +665,7 @@ public class TestHistoryEventTimelineConversion {
     long initedTime = random.nextLong();
     int numTasks = random.nextInt();
     VertexInitializedEvent event = new VertexInitializedEvent(tezVertexID, "v1", initRequestedTime,
-        initedTime, numTasks, "proc", null);
+        initedTime, numTasks, "proc", null, null);
 
     TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
     Assert.assertEquals(EntityTypes.TEZ_VERTEX_ID.name(), timelineEntity.getEntityType());
@@ -935,7 +932,7 @@ public class TestHistoryEventTimelineConversion {
 
   @SuppressWarnings("unchecked")
   @Test(timeout = 5000)
-  public void testConvertVertexParallelismUpdatedEvent() {
+  public void testConvertVertexReconfigreDoneEvent() {
     TezVertexID vId = tezVertexID;
     Map<String, EdgeProperty> edgeMgrs =
         new HashMap<String, EdgeProperty>();
@@ -943,8 +940,8 @@ public class TestHistoryEventTimelineConversion {
     edgeMgrs.put("a", EdgeProperty.create(EdgeManagerPluginDescriptor.create("a.class")
         .setHistoryText("text"), DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
         OutputDescriptor.create("Out"), InputDescriptor.create("In")));
-    VertexParallelismUpdatedEvent event = new VertexParallelismUpdatedEvent(vId, 1, null,
-        edgeMgrs, null, 10);
+    VertexConfigurationDoneEvent event = new VertexConfigurationDoneEvent(vId, 0L, 1, null,
+        edgeMgrs, null, true);
 
     TimelineEntity timelineEntity = HistoryEventTimelineConversion.convertToTimelineEntity(event);
     Assert.assertEquals(ATSConstants.TEZ_VERTEX_ID, timelineEntity.getEntityType());
@@ -959,9 +956,8 @@ public class TestHistoryEventTimelineConversion {
         .contains(tezDAGID.toString()));
 
     TimelineEvent evt = timelineEntity.getEvents().get(0);
-    Assert.assertEquals(HistoryEventType.VERTEX_PARALLELISM_UPDATED.name(), evt.getEventType());
+    Assert.assertEquals(HistoryEventType.VERTEX_CONFIGURE_DONE.name(), evt.getEventType());
     Assert.assertEquals(1, evt.getEventInfo().get(ATSConstants.NUM_TASKS));
-    Assert.assertEquals(10, evt.getEventInfo().get(ATSConstants.OLD_NUM_TASKS));
     Assert.assertNotNull(evt.getEventInfo().get(ATSConstants.UPDATED_EDGE_MANAGERS));
 
     Map<String, Object> updatedEdgeMgrs = (Map<String, Object>)
@@ -976,7 +972,6 @@ public class TestHistoryEventTimelineConversion {
     Assert.assertEquals("a.class", updatedEdgeMgr.get(DAGUtils.EDGE_MANAGER_CLASS_KEY));
 
     Assert.assertEquals(1, timelineEntity.getOtherInfo().get(ATSConstants.NUM_TASKS));
-
   }
 
   @Test(timeout = 5000)

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
----------------------------------------------------------------------
diff --git a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
index b44b7d4..63e2b86 100644
--- a/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
+++ b/tez-runtime-internals/src/main/java/org/apache/tez/runtime/api/impl/TezEvent.java
@@ -278,7 +278,7 @@ public class TezEvent implements Writable {
     } else {
       out.writeBoolean(false);
     }
-  }
+  } 
 
   @Override
   public void readFields(DataInput in) throws IOException {

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-tests/src/test/java/org/apache/tez/test/AMShutdownController.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/AMShutdownController.java b/tez-tests/src/test/java/org/apache/tez/test/AMShutdownController.java
new file mode 100644
index 0000000..4baf6de
--- /dev/null
+++ b/tez-tests/src/test/java/org/apache/tez/test/AMShutdownController.java
@@ -0,0 +1,57 @@
+/**
+ * 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.test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.recovery.RecoveryService;
+
+public abstract class AMShutdownController {
+
+  private List<DAGHistoryEvent> historyEvents = new ArrayList<DAGHistoryEvent>();
+  
+  protected AppContext appContext;
+  protected RecoveryService recoveryService;
+  
+  public AMShutdownController(AppContext appContext, RecoveryService recoveryService) {
+    this.appContext = appContext;
+    this.recoveryService = recoveryService;
+  }
+
+  public void preHandleHistoryEvent(DAGHistoryEvent event) {
+    historyEvents.add(event);
+    if (shouldShutdownPreEvent(event, historyEvents)) {
+      System.exit(1);
+    }
+  }
+
+  public void postHandleHistoryEvent(DAGHistoryEvent event) {
+    if (shouldShutdownPostEvent(event, historyEvents)) {
+      System.exit(1);
+    }
+  }
+
+  protected abstract boolean shouldShutdownPreEvent(DAGHistoryEvent curEvent,
+      List<DAGHistoryEvent> historyEvents);
+
+  protected abstract boolean shouldShutdownPostEvent(DAGHistoryEvent curEvent,
+      List<DAGHistoryEvent> historyEvents);
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java
new file mode 100644
index 0000000..cec8fbd
--- /dev/null
+++ b/tez-tests/src/test/java/org/apache/tez/test/RecoveryServiceWithEventHandlingHook.java
@@ -0,0 +1,386 @@
+/**
+ * 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.test;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.commons.codec.binary.Base64;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.tez.common.ReflectionUtils;
+import org.apache.tez.dag.api.TezReflectionException;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.SummaryEvent;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.history.events.TaskFinishedEvent;
+import org.apache.tez.dag.history.events.TaskStartedEvent;
+import org.apache.tez.dag.history.events.VertexFinishedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
+import org.apache.tez.dag.history.events.VertexStartedEvent;
+import org.apache.tez.dag.history.recovery.RecoveryService;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.test.RecoveryServiceWithEventHandlingHook.SimpleShutdownCondition.TIMING;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Add hook before/after processing RecoveryEvent & SummaryEvent
+ *
+ */
+public class RecoveryServiceWithEventHandlingHook extends RecoveryService {
+
+  public static final String AM_RECOVERY_SERVICE_HOOK_CLASS = "tez.test.am.recovery_service.hook";
+  private static final Logger LOG = LoggerFactory.getLogger(RecoveryServiceWithEventHandlingHook.class);
+  private RecoveryServiceHook hook;
+  private boolean shutdownInvoked = false;
+  public RecoveryServiceWithEventHandlingHook(AppContext appContext) {
+    super(appContext);
+  }
+
+  @Override
+  public void serviceInit(Configuration conf) throws Exception {
+    super.serviceInit(conf);
+    String clazz = conf.get(AM_RECOVERY_SERVICE_HOOK_CLASS);
+    Preconditions.checkArgument(clazz != null, "RecoveryServiceHook class is not specified");
+    this.hook = ReflectionUtils.createClazzInstance(clazz, 
+        new Class[]{RecoveryServiceWithEventHandlingHook.class, AppContext.class},
+        new Object[]{this, super.appContext});
+  }
+
+  @Override
+  protected void handleRecoveryEvent(DAGHistoryEvent event) throws IOException {
+    hook.preHandleRecoveryEvent(event);
+    if (shutdownInvoked) {
+      return;
+    }
+    super.handleRecoveryEvent(event);
+    hook.postHandleRecoveryEvent(event);
+  }
+
+  @Override
+  protected void handleSummaryEvent(TezDAGID dagID, HistoryEventType eventType,
+      SummaryEvent summaryEvent) throws IOException {
+    hook.preHandleSummaryEvent(eventType, summaryEvent);
+    if (shutdownInvoked) {
+      return;
+    }
+    super.handleSummaryEvent(dagID, eventType, summaryEvent);
+    hook.postHandleSummaryEvent(eventType, summaryEvent);
+  }
+
+  private void shutdown() {
+    // start a new thread to shutdown AM otherwise will cause dead lock
+    // (JVM exit will DAGAppMasterShutdownHook called and RecoveryService's stop will be called
+    // which will drain all the events)
+    Thread shutdownThread = new Thread("AMShutdown Thread") {
+      @Override
+      public void run() {
+        LOG.info("Try to kill AM");
+        System.exit(1);
+      }
+    };
+    // stop process recovery events
+    super.setStopped(true);
+    shutdownInvoked = true;
+    shutdownThread.start();
+  }
+
+  /**
+   * Abstract class to allow do something before/after processing recovery events
+   *
+   */
+  public static abstract class RecoveryServiceHook {
+
+    protected RecoveryServiceWithEventHandlingHook recoveryService;
+    protected AppContext appContext;
+
+    public RecoveryServiceHook(RecoveryServiceWithEventHandlingHook recoveryService, AppContext appContext) {
+      this.recoveryService = recoveryService;
+      this.appContext = appContext;
+    }
+
+    public abstract void preHandleRecoveryEvent(DAGHistoryEvent event) throws IOException;
+
+    public abstract void postHandleRecoveryEvent(DAGHistoryEvent event) throws IOException;
+
+    public abstract void preHandleSummaryEvent(HistoryEventType eventType,
+        SummaryEvent summaryEvent) throws IOException;
+
+    public abstract void postHandleSummaryEvent(HistoryEventType eventType,
+        SummaryEvent summaryEvent) throws IOException;
+
+  }
+
+  /**
+   * Shutdown AM before/after a specified recovery event is processed.
+   * Only do it in the first AM attempt
+   *
+   */
+  public static class SimpleRecoveryEventHook extends RecoveryServiceHook {
+
+    public static final String SIMPLE_SHUTDOWN_CONDITION = "tez.test.recovery.simple_shutdown_condition";
+    private SimpleShutdownCondition shutdownCondition;
+
+    public SimpleRecoveryEventHook(
+        RecoveryServiceWithEventHandlingHook recoveryService, AppContext appContext) {
+      super(recoveryService, appContext);
+      this.shutdownCondition = new SimpleShutdownCondition();
+      try {
+        Preconditions.checkArgument(recoveryService.getConfig().get(SIMPLE_SHUTDOWN_CONDITION) != null,
+            SIMPLE_SHUTDOWN_CONDITION + " is not set in TezConfiguration");
+        this.shutdownCondition.deserialize(recoveryService.getConfig().get(SIMPLE_SHUTDOWN_CONDITION));
+      } catch (IOException e) {
+        throw new TezUncheckedException("Can not initialize SimpleShutdownCondition", e);
+      }
+    }
+
+    @Override
+    public void preHandleRecoveryEvent(DAGHistoryEvent event)
+        throws IOException {
+      if (shutdownCondition.timing.equals(TIMING.PRE)
+          && appContext.getApplicationAttemptId().getAttemptId() == 1
+          && shouldShutdown(event)) {
+        recoveryService.shutdown();
+      }
+    }
+
+    @Override
+    public void postHandleRecoveryEvent(DAGHistoryEvent event)
+        throws IOException {
+      if (shutdownCondition.timing.equals(TIMING.POST)
+          && appContext.getApplicationAttemptId().getAttemptId() == 1
+          && shouldShutdown(event)) {
+        recoveryService.shutdown();
+      }
+    }
+
+    private boolean shouldShutdown(DAGHistoryEvent event) {
+      // only check whether to shutdown when it is the first AM attempt
+      if (appContext.getApplicationAttemptId().getAttemptId() >= 2) {
+        return false;
+      }
+      return shutdownCondition.match(event.getHistoryEvent());
+    }
+ 
+    @Override
+    public void preHandleSummaryEvent(HistoryEventType eventType,
+        SummaryEvent summaryEvent) throws IOException {
+    }
+
+    @Override
+    public void postHandleSummaryEvent(HistoryEventType eventType,
+        SummaryEvent summaryEvent) throws IOException {
+    }
+
+  }
+
+  /**
+   * 
+   * Shutdown AM based on one recovery event if it is matched.
+   * This would be serialized as property of TezConfiguration and deserialized at runtime.
+   */
+  public static class SimpleShutdownCondition {
+
+    public static enum TIMING {
+      PRE, // before the event
+      POST, // after the event
+    }
+
+    private TIMING timing;
+    private HistoryEvent event;
+
+    public SimpleShutdownCondition(TIMING timing, HistoryEvent event) {
+      this.timing = timing;
+      this.event = event;
+    }
+
+    public SimpleShutdownCondition() {
+    }
+
+    private String encodeHistoryEvent(HistoryEvent event) throws IOException {
+      ByteArrayOutputStream out = new ByteArrayOutputStream();
+      event.toProtoStream(out);
+      return event.getClass().getName() + ","
+          + Base64.encodeBase64String(out.toByteArray());
+    }
+
+    private HistoryEvent decodeHistoryEvent(String eventClass, String base64)
+        throws IOException {
+      ByteArrayInputStream in = new ByteArrayInputStream(
+          Base64.decodeBase64(base64));
+      try {
+        HistoryEvent event = ReflectionUtils.createClazzInstance(eventClass);
+        event.fromProtoStream(in);
+        return event;
+      } catch (TezReflectionException e) {
+        throw new IOException(e);
+      }
+    }
+
+    public String serialize() throws IOException {
+      StringBuilder builder = new StringBuilder();
+      builder.append(timing.name() + ",");
+      builder.append(encodeHistoryEvent(event));
+      return builder.toString();
+    }
+
+    public SimpleShutdownCondition deserialize(String str) throws IOException {
+      String[] tokens = str.split(",");
+      timing = TIMING.valueOf(tokens[0]);
+      this.event = decodeHistoryEvent(tokens[1], tokens[2]);
+      return this;
+    }
+
+    public HistoryEvent getEvent() {
+      return event;
+    }
+
+    public TIMING getTiming() {
+      return timing;
+    }
+
+    public boolean match(HistoryEvent incomingEvent) {
+      switch (event.getEventType()) {
+      case DAG_SUBMITTED:
+        if (incomingEvent.getEventType() == HistoryEventType.DAG_SUBMITTED) {
+          // only compare eventType
+          return true;
+        }
+        break;
+
+      case DAG_INITIALIZED:
+        if (incomingEvent.getEventType() == HistoryEventType.DAG_INITIALIZED) {
+          // only compare eventType
+          return true;
+        }
+        break;
+
+      case DAG_STARTED:
+        if (incomingEvent.getEventType() == HistoryEventType.DAG_STARTED) {
+          // only compare eventType
+          return true;
+        }
+        break;
+
+      case DAG_FINISHED:
+        if (incomingEvent.getEventType() == HistoryEventType.DAG_FINISHED) {
+          // only compare eventType
+          return true;
+        }
+        break;
+
+      case VERTEX_INITIALIZED:
+        if (incomingEvent.getEventType() == HistoryEventType.VERTEX_INITIALIZED) {
+          VertexInitializedEvent otherEvent = (VertexInitializedEvent) incomingEvent;
+          VertexInitializedEvent conditionEvent = (VertexInitializedEvent) event;
+          // compare vertexId;
+          return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId();
+        }
+        break;
+
+      case VERTEX_STARTED:
+        if (incomingEvent.getEventType() == HistoryEventType.VERTEX_STARTED) {
+          VertexStartedEvent otherEvent = (VertexStartedEvent) incomingEvent;
+          VertexStartedEvent conditionEvent = (VertexStartedEvent) event;
+          // compare vertexId
+          return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId();
+        }
+        break;
+
+      case VERTEX_FINISHED:
+        if (incomingEvent.getEventType() == HistoryEventType.VERTEX_FINISHED) {
+          VertexFinishedEvent otherEvent = (VertexFinishedEvent) incomingEvent;
+          VertexFinishedEvent conditionEvent = (VertexFinishedEvent) event;
+          // compare vertexId
+          return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId();
+        }
+        break;
+      case VERTEX_CONFIGURE_DONE:
+        if (incomingEvent.getEventType() == HistoryEventType.VERTEX_CONFIGURE_DONE) {
+          VertexConfigurationDoneEvent otherEvent = (VertexConfigurationDoneEvent) incomingEvent;
+          VertexConfigurationDoneEvent conditionEvent = (VertexConfigurationDoneEvent) event;
+          // compare vertexId
+          return otherEvent.getVertexID().getId() == conditionEvent.getVertexID().getId();
+        }
+        break;
+      case TASK_STARTED:
+        if (incomingEvent.getEventType() == HistoryEventType.TASK_STARTED) {
+          TaskStartedEvent otherEvent = (TaskStartedEvent) incomingEvent;
+          TaskStartedEvent conditionEvent = (TaskStartedEvent) event;
+          // compare vertexId and taskId
+          return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId()
+              && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId();
+        }
+        break;
+
+      case TASK_FINISHED:
+        if (incomingEvent.getEventType() == HistoryEventType.TASK_FINISHED) {
+          TaskFinishedEvent otherEvent = (TaskFinishedEvent) incomingEvent;
+          TaskFinishedEvent conditionEvent = (TaskFinishedEvent) event;
+          // compare vertexId and taskId
+          return otherEvent.getTaskID().getVertexID().getId() == conditionEvent.getTaskID().getVertexID().getId()
+              && otherEvent.getTaskID().getId() == conditionEvent.getTaskID().getId();
+        }
+        break;
+
+      case TASK_ATTEMPT_STARTED:
+        if (incomingEvent.getEventType() == HistoryEventType.TASK_ATTEMPT_STARTED) {
+          TaskAttemptStartedEvent otherEvent = (TaskAttemptStartedEvent) incomingEvent;
+          TaskAttemptStartedEvent conditionEvent = (TaskAttemptStartedEvent) event;
+          // compare vertexId, taskId & taskAttemptId
+          return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() 
+              == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId()
+              && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId()
+              && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId();
+        }
+        break;
+
+      case TASK_ATTEMPT_FINISHED:
+        if (incomingEvent.getEventType() == HistoryEventType.TASK_ATTEMPT_FINISHED) {
+          TaskAttemptFinishedEvent otherEvent = (TaskAttemptFinishedEvent) incomingEvent;
+          TaskAttemptFinishedEvent conditionEvent = (TaskAttemptFinishedEvent) event;
+          // compare vertexId, taskId & taskAttemptId
+          return otherEvent.getTaskAttemptID().getTaskID().getVertexID().getId() 
+              == conditionEvent.getTaskAttemptID().getTaskID().getVertexID().getId()
+              && otherEvent.getTaskAttemptID().getTaskID().getId() == conditionEvent.getTaskAttemptID().getTaskID().getId()
+              && otherEvent.getTaskAttemptID().getId() == conditionEvent.getTaskAttemptID().getId();
+        }
+        break;
+      default:
+        LOG.info("do nothing with event:"
+            + event.getEventType());
+      }
+
+      return false;
+    }
+    
+    public HistoryEventType getEventType() {
+      return event.getEventType();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
index 778825b..8e41b7e 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery.java
@@ -24,26 +24,18 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.tez.client.TezClientUtils;
 import org.apache.tez.client.TezClient;
-import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.dag.api.DAG;
 import org.apache.tez.dag.api.DataSourceDescriptor;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.InputInitializerDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.api.client.DAGStatus.State;
-import org.apache.tez.dag.app.RecoveryParser;
-import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.HistoryEventType;
-import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
 import org.apache.tez.test.dag.MultiAttemptDAG;
 import org.apache.tez.test.dag.MultiAttemptDAG.FailingInputInitializer;
 import org.apache.tez.test.dag.MultiAttemptDAG.NoOpInput;
@@ -57,8 +49,6 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Random;
 
 public class TestDAGRecovery {
@@ -176,56 +166,6 @@ public class TestDAGRecovery {
     Assert.assertEquals(finalState, dagStatus.getState());
   }
 
-  private void verifyRecoveryLog() throws IOException{
-    ApplicationId appId = tezSession.getAppMasterApplicationId();
-    Path tezSystemStagingDir = TezCommonUtils.getTezSystemStagingPath(tezConf, appId.toString());
-    Path recoveryDataDir = TezCommonUtils.getRecoveryPath(tezSystemStagingDir, tezConf);
-
-    FileSystem fs = tezSystemStagingDir.getFileSystem(tezConf);
-    // verify recovery logs in each attempt
-    for (int attemptNum=1; attemptNum<=3; ++attemptNum) {
-      List<HistoryEvent> historyEvents = new ArrayList<HistoryEvent>();
-      // read the recovery logs for current attempt
-      // since dag recovery logs is dispersed in each attempt's recovery directory,
-      // so need to read recovery logs from the first attempt to current attempt
-      for (int i=1 ;i<=attemptNum;++i) {
-        Path currentAttemptRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir,i);
-        Path recoveryFilePath = new Path(currentAttemptRecoveryDataDir,
-        appId.toString().replace("application", "dag") + "_1" + TezConstants.DAG_RECOVERY_RECOVER_FILE_SUFFIX);
-        historyEvents.addAll(RecoveryParser.parseDAGRecoveryFile(
-            fs.open(recoveryFilePath)));
-      }
-
-      int inputInfoEventIndex = -1;
-      int vertexInitedEventIndex = -1;
-      for (int j=0;j<historyEvents.size(); ++j) {
-        HistoryEvent historyEvent = historyEvents.get(j);
-        LOG.info("Parsed event from recovery stream"
-            + ", eventType=" + historyEvent.getEventType()
-            + ", event=" + historyEvent);
-        if (historyEvent.getEventType() ==  HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED) {
-          VertexRecoverableEventsGeneratedEvent dmEvent =
-              (VertexRecoverableEventsGeneratedEvent) historyEvent;
-          // TODO do not need to check whether it is -1 after Tez-1521 is resolved
-          if (dmEvent.getVertexID().getId() == 0 && inputInfoEventIndex == -1) {
-            inputInfoEventIndex = j;
-          }
-        }
-        if (historyEvent.getEventType() == HistoryEventType.VERTEX_INITIALIZED) {
-          VertexInitializedEvent vInitedEvent = (VertexInitializedEvent) historyEvent;
-          if (vInitedEvent.getVertexID().getId() == 0) {
-            vertexInitedEventIndex = j;
-          }
-        }
-      }
-      // v1's init events must be logged before its VertexInitializedEvent (Tez-1345)
-      Assert.assertTrue("can not find VERTEX_DATA_MOVEMENT_EVENTS_GENERATED for v1", inputInfoEventIndex != -1);
-      Assert.assertTrue("can not find VERTEX_INITIALIZED for v1", vertexInitedEventIndex != -1);
-      Assert.assertTrue("VERTEX_DATA_MOVEMENT_EVENTS_GENERATED is logged before VERTEX_INITIALIZED for v1",
-          inputInfoEventIndex < vertexInitedEventIndex);
-    }
-  }
-
   @Test(timeout=120000)
   public void testBasicRecovery() throws Exception {
     DAG dag = MultiAttemptDAG.createDAG("TestBasicRecovery", null);
@@ -236,8 +176,6 @@ public class TestDAGRecovery {
     dag.getVertex("v1").addDataSource("Input", dataSource);
 
     runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
-
-    verifyRecoveryLog();
   }
 
   @Test(timeout=120000)

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java
new file mode 100644
index 0000000..45582a1
--- /dev/null
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestRecovery.java
@@ -0,0 +1,484 @@
+/**
+ * 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.test;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.BufferedReader;
+import java.io.BufferedWriter;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.PathFilter;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
+import org.apache.hadoop.yarn.conf.YarnConfiguration;
+import org.apache.tez.common.counters.TezCounters;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.oldrecords.TaskState;
+import org.apache.tez.dag.app.RecoveryParser;
+import org.apache.tez.dag.app.dag.DAGState;
+import org.apache.tez.dag.app.dag.VertexState;
+import org.apache.tez.dag.app.dag.impl.VertexStats;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.events.DAGFinishedEvent;
+import org.apache.tez.dag.history.events.DAGInitializedEvent;
+import org.apache.tez.dag.history.events.DAGStartedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.history.events.TaskFinishedEvent;
+import org.apache.tez.dag.history.events.TaskStartedEvent;
+import org.apache.tez.dag.history.events.VertexFinishedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
+import org.apache.tez.dag.history.events.VertexStartedEvent;
+import org.apache.tez.dag.history.recovery.RecoveryService;
+import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager;
+import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.examples.HashJoinExample;
+import org.apache.tez.examples.OrderedWordCount;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.test.RecoveryServiceWithEventHandlingHook.SimpleRecoveryEventHook;
+import org.apache.tez.test.RecoveryServiceWithEventHandlingHook.SimpleShutdownCondition;
+import org.apache.tez.test.RecoveryServiceWithEventHandlingHook.SimpleShutdownCondition.TIMING;
+import org.junit.AfterClass;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.collect.Lists;
+
+public class TestRecovery {
+
+  private static final Logger LOG = LoggerFactory.getLogger(TestRecovery.class);
+
+  private static Configuration conf = new Configuration();
+  private static MiniTezCluster miniTezCluster = null;
+  private static String TEST_ROOT_DIR = "target" + Path.SEPARATOR
+      + TestRecovery.class.getName() + "-tmpDir";
+  private static MiniDFSCluster dfsCluster = null;
+  private static FileSystem remoteFs = null;
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    LOG.info("Starting mini clusters");
+    try {
+      conf.set(MiniDFSCluster.HDFS_MINIDFS_BASEDIR, TEST_ROOT_DIR);
+      dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(3)
+          .format(true).racks(null).build();
+      remoteFs = dfsCluster.getFileSystem();
+    } catch (IOException io) {
+      throw new RuntimeException("problem starting mini dfs cluster", io);
+    }
+    if (miniTezCluster == null) {
+      miniTezCluster = new MiniTezCluster(TestRecovery.class.getName(), 1, 1, 1);
+      Configuration miniTezconf = new Configuration(conf);
+      miniTezconf.setInt(YarnConfiguration.RM_AM_MAX_ATTEMPTS, 4);
+      miniTezconf.set("fs.defaultFS", remoteFs.getUri().toString()); // use HDFS
+      miniTezCluster.init(miniTezconf);
+      miniTezCluster.start();
+    }
+  }
+
+  @AfterClass
+  public static void afterClass() throws InterruptedException {
+    if (miniTezCluster != null) {
+      try {
+        LOG.info("Stopping MiniTezCluster");
+        miniTezCluster.stop();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+    if (dfsCluster != null) {
+      try {
+        LOG.info("Stopping DFSCluster");
+        dfsCluster.shutdown();
+      } catch (Exception e) {
+        e.printStackTrace();
+      }
+    }
+  }
+
+  @Test(timeout=1800000)
+  public void testRecovery_OrderedWordCount() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
+        1);
+    TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+    TezVertexID vertexId0 = TezVertexID.getInstance(dagId, 0);
+    TezVertexID vertexId1 = TezVertexID.getInstance(dagId, 1);
+    TezVertexID vertexId2 = TezVertexID.getInstance(dagId, 2);
+    ContainerId containerId = ContainerId.newContainerId(
+        ApplicationAttemptId.newInstance(appId, 1), 1);
+    NodeId nodeId = NodeId.newInstance("localhost", 10);
+    
+    List<TezEvent> initGeneratedEvents = Lists.newArrayList(
+            new TezEvent(InputDataInformationEvent.createWithObjectPayload(0, new Object()), null));
+
+    List<SimpleShutdownCondition> shutdownConditions = Lists
+        .newArrayList(
+            new SimpleShutdownCondition(TIMING.POST, new DAGInitializedEvent(
+                dagId, 0L, "username", "dagName", null)),
+            new SimpleShutdownCondition(TIMING.POST, new DAGStartedEvent(dagId,
+                0L, "username", "dagName")),
+            new SimpleShutdownCondition(TIMING.POST, new DAGFinishedEvent(
+                dagId, 0L, 0L, DAGState.SUCCEEDED, "", new TezCounters(),
+                "username", "dagName", new HashMap<String, Integer>(),
+                ApplicationAttemptId.newInstance(appId, 1))),
+            new SimpleShutdownCondition(TIMING.POST,
+                new VertexInitializedEvent(vertexId0, "Tokenizer", 0L, 0L, 0,
+                    "", null, initGeneratedEvents)),
+            new SimpleShutdownCondition(TIMING.POST,
+                new VertexInitializedEvent(vertexId1, "Summation", 0L, 0L, 0,
+                    "", null, null)),
+            new SimpleShutdownCondition(TIMING.POST,
+                new VertexInitializedEvent(vertexId2, "Sorter", 0L, 0L, 0, "",
+                    null, null)),
+
+            new SimpleShutdownCondition(TIMING.POST,
+                new VertexConfigurationDoneEvent(vertexId0, 0L, 2, null, null,
+                    null, true)),
+                        
+            new SimpleShutdownCondition(TIMING.POST,
+                new VertexConfigurationDoneEvent(vertexId1, 0L, 2, null, null,
+                    null, true)),
+
+            new SimpleShutdownCondition(TIMING.POST,
+                new VertexConfigurationDoneEvent(vertexId2, 0L, 2, null, null,
+                    null, true)),
+            new SimpleShutdownCondition(TIMING.POST, new VertexStartedEvent(
+                vertexId0, 0L, 0L)),
+            new SimpleShutdownCondition(TIMING.POST, new VertexStartedEvent(
+                vertexId1, 0L, 0L)),
+            new SimpleShutdownCondition(TIMING.POST, new VertexStartedEvent(
+                vertexId2, 0L, 0L)),
+
+            new SimpleShutdownCondition(TIMING.POST, new VertexFinishedEvent(
+                vertexId0, "vertexName", 1, 0L, 0L, 0L, 0L, 0L,
+                VertexState.SUCCEEDED, "", new TezCounters(),
+                new VertexStats(), new HashMap<String, Integer>())),
+            new SimpleShutdownCondition(TIMING.POST, new VertexFinishedEvent(
+                vertexId1, "vertexName", 1, 0L, 0L, 0L, 0L, 0L,
+                VertexState.SUCCEEDED, "", new TezCounters(),
+                new VertexStats(), new HashMap<String, Integer>())),
+            new SimpleShutdownCondition(TIMING.POST, new VertexFinishedEvent(
+                vertexId2, "vertexName", 1, 0L, 0L, 0L, 0L, 0L,
+                VertexState.SUCCEEDED, "", new TezCounters(),
+                new VertexStats(), new HashMap<String, Integer>())),
+
+            new SimpleShutdownCondition(TIMING.POST, new TaskStartedEvent(
+                TezTaskID.getInstance(vertexId0, 0), "vertexName", 0L, 0L)),
+            new SimpleShutdownCondition(TIMING.POST, new TaskStartedEvent(
+                TezTaskID.getInstance(vertexId1, 0), "vertexName", 0L, 0L)),
+            new SimpleShutdownCondition(TIMING.POST, new TaskStartedEvent(
+                TezTaskID.getInstance(vertexId2, 0), "vertexName", 0L, 0L)),
+
+            new SimpleShutdownCondition(TIMING.POST, new TaskFinishedEvent(
+                TezTaskID.getInstance(vertexId0, 0), "vertexName", 0L, 0L,
+                null, TaskState.SUCCEEDED, "", new TezCounters(), 0)),
+            new SimpleShutdownCondition(TIMING.POST, new TaskFinishedEvent(
+                TezTaskID.getInstance(vertexId1, 0), "vertexName", 0L, 0L,
+                null, TaskState.SUCCEEDED, "", new TezCounters(), 0)),
+            new SimpleShutdownCondition(TIMING.POST, new TaskFinishedEvent(
+                TezTaskID.getInstance(vertexId2, 0), "vertexName", 0L, 0L,
+                null, TaskState.SUCCEEDED, "", new TezCounters(), 0)),
+
+            new SimpleShutdownCondition(TIMING.POST,
+                new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(
+                    TezTaskID.getInstance(vertexId0, 0), 0), "vertexName", 0L,
+                    containerId, nodeId, "", "", "")),
+            new SimpleShutdownCondition(TIMING.POST,
+                new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(
+                    TezTaskID.getInstance(vertexId1, 0), 0), "vertexName", 0L,
+                    containerId, nodeId, "", "", "")),
+            new SimpleShutdownCondition(TIMING.POST,
+                new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(
+                    TezTaskID.getInstance(vertexId2, 0), 0), "vertexName", 0L,
+                    containerId, nodeId, "", "", ""))
+
+        );
+
+    Random rand = new Random();
+    for (int i = 0; i < shutdownConditions.size(); i++) {
+      // randomly choose half of the test scenario to avoid
+      // timeout.
+      if (rand.nextDouble() < 0.5) {
+        testOrderedWordCount(shutdownConditions.get(i), true);
+      }
+    }
+  }
+
+  private void testOrderedWordCount(SimpleShutdownCondition shutdownCondition,
+      boolean enableAutoParallelism) throws Exception {
+    LOG.info("shutdownCondition:" + shutdownCondition.getEventType()
+        + ", event=" + shutdownCondition.getEvent());
+    String inputDirStr = "/tmp/owc-input/";
+    Path inputDir = new Path(inputDirStr);
+    Path stagingDirPath = new Path("/tmp/owc-staging-dir");
+    remoteFs.mkdirs(inputDir);
+    remoteFs.mkdirs(stagingDirPath);
+    TestTezJobs.generateOrderedWordCountInput(inputDir, remoteFs);
+
+    String outputDirStr = "/tmp/owc-output/";
+    Path outputDir = new Path(outputDirStr);
+
+    TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+    tezConf.setInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, 4);
+    tezConf.set(TezConfiguration.TEZ_AM_RECOVERY_SERVICE_CLASS,
+        RecoveryServiceWithEventHandlingHook.class.getName());
+    tezConf.set(
+        RecoveryServiceWithEventHandlingHook.AM_RECOVERY_SERVICE_HOOK_CLASS,
+        SimpleRecoveryEventHook.class.getName());
+    tezConf.set(SimpleRecoveryEventHook.SIMPLE_SHUTDOWN_CONDITION,
+        shutdownCondition.serialize());
+    tezConf.setBoolean(
+        ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL,
+        enableAutoParallelism);
+    tezConf.setBoolean(
+        RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, false);
+    tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirPath.toString());
+    tezConf.setBoolean(
+        TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, false);
+    tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "INFO;org.apache.tez=DEBUG");
+
+    OrderedWordCount job = new OrderedWordCount();
+    Assert
+        .assertTrue("OrderedWordCount failed", job.run(tezConf, new String[] {
+            inputDirStr, outputDirStr, "5" }, null) == 0);
+    TestTezJobs.verifyOutput(outputDir, remoteFs);
+    List<HistoryEvent> historyEventsOfAttempt1 = RecoveryParser
+        .readRecoveryEvents(tezConf, job.getAppId(), 1);
+    HistoryEvent lastEvent = historyEventsOfAttempt1
+        .get(historyEventsOfAttempt1.size() - 1);
+    assertEquals(shutdownCondition.getEvent().getEventType(),
+        lastEvent.getEventType());
+    assertTrue(shutdownCondition.match(lastEvent));
+
+  }
+
+  @Test(timeout = 1800000)
+  public void testRecovery_HashJoin() throws Exception {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(),
+        1);
+    TezDAGID dagId = TezDAGID.getInstance(appId, 1);
+    TezVertexID vertexId0 = TezVertexID.getInstance(dagId, 0);
+    TezVertexID vertexId1 = TezVertexID.getInstance(dagId, 1);
+    TezVertexID vertexId2 = TezVertexID.getInstance(dagId, 2);
+    ContainerId containerId = ContainerId.newContainerId(
+        ApplicationAttemptId.newInstance(appId, 1), 1);
+    NodeId nodeId = NodeId.newInstance("localhost", 10);
+    List<TezEvent> initGeneratedEvents = Lists.newArrayList(
+        new TezEvent(InputDataInformationEvent.createWithObjectPayload(0, new Object()), null));
+
+    List<SimpleShutdownCondition> shutdownConditions = Lists.newArrayList(
+
+        new SimpleShutdownCondition(TIMING.POST, new DAGInitializedEvent(dagId,
+            0L, "username", "dagName", null)),
+        new SimpleShutdownCondition(TIMING.POST, new DAGStartedEvent(dagId, 0L,
+            "username", "dagName")),
+        new SimpleShutdownCondition(TIMING.POST, new DAGFinishedEvent(dagId,
+            0L, 0L, DAGState.SUCCEEDED, "", new TezCounters(), "username",
+            "dagName", new HashMap<String, Integer>(), ApplicationAttemptId
+                .newInstance(appId, 1))),
+        new SimpleShutdownCondition(TIMING.POST, new VertexInitializedEvent(
+            vertexId0, "hashSide", 0L, 0L, 0, "", null, initGeneratedEvents)),
+        new SimpleShutdownCondition(TIMING.POST, new VertexInitializedEvent(
+            vertexId1, "streamingSide", 0L, 0L, 0, "", null, null)),
+        new SimpleShutdownCondition(TIMING.POST, new VertexInitializedEvent(
+            vertexId2, "joiner", 0L, 0L, 0, "", null, null)),
+
+        new SimpleShutdownCondition(TIMING.POST, new VertexStartedEvent(
+            vertexId0, 0L, 0L)),
+        new SimpleShutdownCondition(TIMING.POST, new VertexStartedEvent(
+            vertexId1, 0L, 0L)),
+        new SimpleShutdownCondition(TIMING.POST, new VertexStartedEvent(
+            vertexId2, 0L, 0L)),
+
+        new SimpleShutdownCondition(TIMING.POST,
+            new VertexConfigurationDoneEvent(vertexId0, 0L, 2, null, null,
+                null, true)),
+                    
+        new SimpleShutdownCondition(TIMING.POST,
+            new VertexConfigurationDoneEvent(vertexId1, 0L, 2, null, null,
+                null, true)),
+
+        new SimpleShutdownCondition(TIMING.POST,
+            new VertexConfigurationDoneEvent(vertexId2, 0L, 2, null, null,
+                null, true)),
+                    
+        new SimpleShutdownCondition(TIMING.POST, new VertexFinishedEvent(
+            vertexId0, "vertexName", 1, 0L, 0L, 0L, 0L, 0L,
+            VertexState.SUCCEEDED, "", new TezCounters(), new VertexStats(),
+            new HashMap<String, Integer>())),
+        new SimpleShutdownCondition(TIMING.POST, new VertexFinishedEvent(
+            vertexId1, "vertexName", 1, 0L, 0L, 0L, 0L, 0L,
+            VertexState.SUCCEEDED, "", new TezCounters(), new VertexStats(),
+            new HashMap<String, Integer>())),
+        new SimpleShutdownCondition(TIMING.POST, new VertexFinishedEvent(
+            vertexId2, "vertexName", 1, 0L, 0L, 0L, 0L, 0L,
+            VertexState.SUCCEEDED, "", new TezCounters(), new VertexStats(),
+            new HashMap<String, Integer>())),
+
+        new SimpleShutdownCondition(TIMING.POST, new TaskStartedEvent(TezTaskID
+            .getInstance(vertexId0, 0), "vertexName", 0L, 0L)),
+        new SimpleShutdownCondition(TIMING.POST, new TaskStartedEvent(TezTaskID
+            .getInstance(vertexId1, 0), "vertexName", 0L, 0L)),
+        new SimpleShutdownCondition(TIMING.POST, new TaskStartedEvent(TezTaskID
+            .getInstance(vertexId2, 0), "vertexName", 0L, 0L)),
+
+        new SimpleShutdownCondition(TIMING.POST, new TaskFinishedEvent(
+            TezTaskID.getInstance(vertexId0, 0), "vertexName", 0L, 0L, null,
+            TaskState.SUCCEEDED, "", new TezCounters(), 0)),
+        new SimpleShutdownCondition(TIMING.POST, new TaskFinishedEvent(
+            TezTaskID.getInstance(vertexId1, 0), "vertexName", 0L, 0L, null,
+            TaskState.SUCCEEDED, "", new TezCounters(), 0)),
+        new SimpleShutdownCondition(TIMING.POST, new TaskFinishedEvent(
+            TezTaskID.getInstance(vertexId2, 0), "vertexName", 0L, 0L, null,
+            TaskState.SUCCEEDED, "", new TezCounters(), 0)),
+
+        new SimpleShutdownCondition(TIMING.POST,
+            new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(
+                TezTaskID.getInstance(vertexId0, 0), 0), "vertexName", 0L,
+                containerId, nodeId, "", "", "")),
+        new SimpleShutdownCondition(TIMING.POST,
+            new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(
+                TezTaskID.getInstance(vertexId1, 0), 0), "vertexName", 0L,
+                containerId, nodeId, "", "", "")),
+        new SimpleShutdownCondition(TIMING.POST,
+            new TaskAttemptStartedEvent(TezTaskAttemptID.getInstance(
+                TezTaskID.getInstance(vertexId2, 0), 0), "vertexName", 0L,
+                containerId, nodeId, "", "", ""))
+
+    );
+
+    Random rand = new Random();
+    for (int i = 0; i < shutdownConditions.size(); i++) {
+      // randomly choose half of the test scenario to avoid
+      // timeout.
+      if (rand.nextDouble() < 0.5) {
+        testHashJoinExample(shutdownConditions.get(i), true);
+      }
+    }
+  }
+
+  private void testHashJoinExample(SimpleShutdownCondition shutdownCondition,
+      boolean enableAutoParallelism) throws Exception {
+    HashJoinExample hashJoinExample = new HashJoinExample();
+    TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+    tezConf.setInt(TezConfiguration.TEZ_AM_MAX_APP_ATTEMPTS, 4);
+    tezConf.set(TezConfiguration.TEZ_AM_RECOVERY_SERVICE_CLASS,
+        RecoveryServiceWithEventHandlingHook.class.getName());
+    tezConf.set(
+        RecoveryServiceWithEventHandlingHook.AM_RECOVERY_SERVICE_HOOK_CLASS,
+        SimpleRecoveryEventHook.class.getName());
+    tezConf.set(SimpleRecoveryEventHook.SIMPLE_SHUTDOWN_CONDITION,
+        shutdownCondition.serialize());
+    tezConf.setBoolean(
+        ShuffleVertexManager.TEZ_SHUFFLE_VERTEX_MANAGER_ENABLE_AUTO_PARALLEL,
+        enableAutoParallelism);
+    tezConf.setBoolean(
+        RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, false);
+    tezConf.setBoolean(
+        TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, false);
+    tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "INFO;org.apache.tez=DEBUG");
+
+    hashJoinExample.setConf(tezConf);
+    Path stagingDirPath = new Path("/tmp/tez-staging-dir");
+    Path inPath1 = new Path("/tmp/hashJoin/inPath1");
+    Path inPath2 = new Path("/tmp/hashJoin/inPath2");
+    Path outPath = new Path("/tmp/hashJoin/outPath");
+    remoteFs.delete(outPath, true);
+    remoteFs.mkdirs(inPath1);
+    remoteFs.mkdirs(inPath2);
+    remoteFs.mkdirs(stagingDirPath);
+
+    Set<String> expectedResult = new HashSet<String>();
+
+    FSDataOutputStream out1 = remoteFs.create(new Path(inPath1, "file"));
+    FSDataOutputStream out2 = remoteFs.create(new Path(inPath2, "file"));
+    BufferedWriter writer1 = new BufferedWriter(new OutputStreamWriter(out1));
+    BufferedWriter writer2 = new BufferedWriter(new OutputStreamWriter(out2));
+    for (int i = 0; i < 20; i++) {
+      String term = "term" + i;
+      writer1.write(term);
+      writer1.newLine();
+      if (i % 2 == 0) {
+        writer2.write(term);
+        writer2.newLine();
+        expectedResult.add(term);
+      }
+    }
+    writer1.close();
+    writer2.close();
+    out1.close();
+    out2.close();
+
+    String[] args = new String[] {
+        "-D" + TezConfiguration.TEZ_AM_STAGING_DIR + "="
+            + stagingDirPath.toString(), inPath1.toString(),
+        inPath2.toString(), "1", outPath.toString() };
+    assertEquals(0, hashJoinExample.run(args));
+
+    FileStatus[] statuses = remoteFs.listStatus(outPath, new PathFilter() {
+      public boolean accept(Path p) {
+        String name = p.getName();
+        return !name.startsWith("_") && !name.startsWith(".");
+      }
+    });
+    assertEquals(1, statuses.length);
+    FSDataInputStream inStream = remoteFs.open(statuses[0].getPath());
+    BufferedReader reader = new BufferedReader(new InputStreamReader(inStream));
+    String line;
+    while ((line = reader.readLine()) != null) {
+      assertTrue(expectedResult.remove(line));
+    }
+    reader.close();
+    inStream.close();
+    assertEquals(0, expectedResult.size());
+
+    List<HistoryEvent> historyEventsOfAttempt1 = RecoveryParser
+        .readRecoveryEvents(tezConf, hashJoinExample.getAppId(), 1);
+    HistoryEvent lastEvent = historyEventsOfAttempt1
+        .get(historyEventsOfAttempt1.size() - 1);
+    assertEquals(shutdownCondition.getEvent().getEventType(),
+        lastEvent.getEventType());
+    assertTrue(shutdownCondition.match(lastEvent));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
index ab89ddb..c3e8487 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestTezJobs.java
@@ -573,7 +573,7 @@ public class TestTezJobs {
     }
   }
 
-  private void generateOrderedWordCountInput(Path inputDir, FileSystem fs) throws IOException {
+  public static void generateOrderedWordCountInput(Path inputDir, FileSystem fs) throws IOException {
     Path dataPath1 = new Path(inputDir, "inPath1");
     Path dataPath2 = new Path(inputDir, "inPath2");
 
@@ -606,7 +606,7 @@ public class TestTezJobs {
     }
   }
 
-  private void verifyOrderedWordCountOutput(Path resultFile, FileSystem fs) throws IOException {
+  public static void verifyOrderedWordCountOutput(Path resultFile, FileSystem fs) throws IOException {
     FSDataInputStream inputStream = fs.open(resultFile);
     final String prefix = "a";
     int currentCounter = 10;
@@ -631,7 +631,7 @@ public class TestTezJobs {
     Assert.assertEquals(0, currentCounter);
   }
   
-  private void verifyOutput(Path outputDir, FileSystem fs) throws IOException {
+  public static void verifyOutput(Path outputDir, FileSystem fs) throws IOException {
     FileStatus[] fileStatuses = fs.listStatus(outputDir);
     Path resultFile = null;
     boolean foundResult = false;

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
index 5c6f855..cdf69e6 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/dag/MultiAttemptDAG.java
@@ -121,7 +121,7 @@ public class MultiAttemptDAG {
             + ", currentAttempt=" + getContext().getDAGAttemptNumber());
         if (successAttemptId > getContext().getDAGAttemptNumber()) {
           Runtime.getRuntime().halt(-1);
-        } else if (successAttemptId == getContext().getDAGAttemptNumber()) {
+        } else {
           LOG.info("Scheduling tasks for vertex=" + getContext().getVertexName());
           int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
           List<ScheduleTaskRequest> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);


[2/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
deleted file mode 100644
index e389d64..0000000
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
+++ /dev/null
@@ -1,1340 +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.dag.impl;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
-import static org.mockito.Mockito.doReturn;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.security.Credentials;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.event.DrainDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.oldrecords.TaskState;
-import org.apache.tez.dag.api.records.DAGProtos;
-import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
-import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
-import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataSourceType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeSchedulingType;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
-import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
-import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
-import org.apache.tez.dag.api.records.DAGProtos.RootInputLeafOutputProto;
-import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
-import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.ClusterInfo;
-import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
-import org.apache.tez.dag.app.TaskHeartbeatHandler;
-import org.apache.tez.dag.app.dag.DAGState;
-import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.dag.VertexState;
-import org.apache.tez.dag.app.dag.VertexTerminationCause;
-import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType;
-import org.apache.tez.dag.app.dag.event.DAGEvent;
-import org.apache.tez.dag.app.dag.event.DAGEventRecoverEvent;
-import org.apache.tez.dag.app.dag.event.DAGEventType;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
-import org.apache.tez.dag.app.dag.event.TaskEvent;
-import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
-import org.apache.tez.dag.app.dag.event.TaskEventType;
-import org.apache.tez.dag.app.dag.event.VertexEvent;
-import org.apache.tez.dag.app.dag.event.VertexEventManagerUserCodeError;
-import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
-import org.apache.tez.dag.app.dag.event.VertexEventType;
-import org.apache.tez.dag.app.dag.impl.AMUserCodeException.Source;
-import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter;
-import org.apache.tez.dag.history.HistoryEventType;
-import org.apache.tez.dag.history.events.DAGInitializedEvent;
-import org.apache.tez.dag.history.events.DAGStartedEvent;
-import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
-import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
-import org.apache.tez.dag.history.events.VertexFinishedEvent;
-import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexStartedEvent;
-import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexFinishStateProto;
-import org.apache.tez.runtime.api.OutputCommitter;
-import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import org.apache.tez.runtime.api.impl.EventMetaData;
-import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
-import org.apache.tez.runtime.api.impl.TezEvent;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-public class TestVertexRecovery {
-
-  private static final Logger LOG = LoggerFactory.getLogger(TestVertexRecovery.class);
-
-  private DrainDispatcher dispatcher;
-
-  private AppContext mockAppContext;
-  private ApplicationId appId = ApplicationId.newInstance(
-      System.currentTimeMillis(), 1);
-  private DAGImpl dag;
-  private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
-  private String user = "user";
-
-  private long initRequestedTime = 100L;
-  private long initedTime = initRequestedTime + 100L;
-
-  /*
-   * v1 v2 \ / v3
-   */
-  private DAGPlan createDAGPlan() {
-    DAGPlan dag =
-        DAGPlan
-            .newBuilder()
-            .setName("testverteximpl")
-            .addVertex(
-                VertexPlan
-                    .newBuilder()
-                    .setName("vertex1")
-                    .setType(PlanVertexType.NORMAL)
-                    .addTaskLocationHint(
-                        PlanTaskLocationHint.newBuilder().addHost("host1")
-                            .addRack("rack1").build())
-                    .setTaskConfig(
-                        PlanTaskConfiguration.newBuilder().setNumTasks(1)
-                            .setVirtualCores(4).setMemoryMb(1024)
-                            .setJavaOpts("").setTaskModule("x1.y1").build())
-                    .addOutEdgeId("e1")
-                    .addOutputs(
-                        DAGProtos.RootInputLeafOutputProto
-                            .newBuilder()
-                            .setIODescriptor(
-                                TezEntityDescriptorProto.newBuilder()
-                                    .setClassName("output").build())
-                            .setName("outputx")
-                            .setControllerDescriptor(
-                                TezEntityDescriptorProto
-                                    .newBuilder()
-                                    .setClassName(
-                                        CountingOutputCommitter.class.getName())))
-                    .build())
-            .addVertex(
-                VertexPlan
-                    .newBuilder()
-                    .setName("vertex2")
-                    .setType(PlanVertexType.NORMAL)
-                    .addTaskLocationHint(
-                        PlanTaskLocationHint.newBuilder().addHost("host2")
-                            .addRack("rack2").build())
-                    .setTaskConfig(
-                        PlanTaskConfiguration.newBuilder().setNumTasks(2)
-                            .setVirtualCores(4).setMemoryMb(1024)
-                            .setJavaOpts("").setTaskModule("x2.y2").build())
-                    .addOutEdgeId("e2").build())
-            .addVertex(
-                VertexPlan
-                    .newBuilder()
-                    .setName("vertex3")
-                    .setType(PlanVertexType.NORMAL)
-                    .setProcessorDescriptor(
-                        TezEntityDescriptorProto.newBuilder().setClassName(
-                            "x3.y3"))
-                    .addTaskLocationHint(
-                        PlanTaskLocationHint.newBuilder().addHost("host3")
-                            .addRack("rack3").build())
-                    .setTaskConfig(
-                        PlanTaskConfiguration.newBuilder().setNumTasks(2)
-                            .setVirtualCores(4).setMemoryMb(1024)
-                            .setJavaOpts("foo").setTaskModule("x3.y3").build())
-                    .addInEdgeId("e1")
-                    .addInEdgeId("e2")
-                    .addOutputs(
-                        DAGProtos.RootInputLeafOutputProto
-                            .newBuilder()
-                            .setIODescriptor(
-                                TezEntityDescriptorProto.newBuilder()
-                                    .setClassName("output").build())
-                            .setName("outputx")
-                            .setControllerDescriptor(
-                                TezEntityDescriptorProto
-                                    .newBuilder()
-                                    .setClassName(
-                                        CountingOutputCommitter.class.getName())))
-                    .build()
-
-            )
-
-            .addEdge(
-                EdgePlan
-                    .newBuilder()
-                    .setEdgeDestination(
-                        TezEntityDescriptorProto.newBuilder().setClassName(
-                            "i3_v1"))
-                    .setInputVertexName("vertex1")
-                    .setEdgeSource(
-                        TezEntityDescriptorProto.newBuilder()
-                            .setClassName("o1"))
-                    .setOutputVertexName("vertex3")
-                    .setDataMovementType(
-                        PlanEdgeDataMovementType.SCATTER_GATHER).setId("e1")
-                    .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                    .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                    .build())
-            .addEdge(
-                EdgePlan
-                    .newBuilder()
-                    .setEdgeDestination(
-                        TezEntityDescriptorProto.newBuilder().setClassName(
-                            "i3_v2"))
-                    .setInputVertexName("vertex2")
-                    .setEdgeSource(
-                        TezEntityDescriptorProto.newBuilder()
-                            .setClassName("o2"))
-                    .setOutputVertexName("vertex3")
-                    .setDataMovementType(
-                        PlanEdgeDataMovementType.SCATTER_GATHER).setId("e2")
-                    .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                    .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                    .build()).build();
-
-    return dag;
-  }
-
-  private DAGPlan createDAGPlanSingleVertex() {
-    DAGPlan dag =
-        DAGPlan
-            .newBuilder()
-            .setName("testverteximpl")
-            .addVertex(
-                VertexPlan
-                    .newBuilder()
-                    .setName("vertex1")
-                    .setType(PlanVertexType.NORMAL)
-                    .addTaskLocationHint(
-                        PlanTaskLocationHint.newBuilder().addHost("host1")
-                            .addRack("rack1").build())
-                    .setTaskConfig(
-                        PlanTaskConfiguration.newBuilder().setNumTasks(-1)
-                            .setVirtualCores(4).setMemoryMb(1024)
-                            .setJavaOpts("").setTaskModule("x1.y1").build())
-                    .addInputs(RootInputLeafOutputProto.newBuilder()
-                            .setIODescriptor(
-                                TezEntityDescriptorProto.newBuilder()
-                                    .setClassName("input").build())
-                            .setName("inputx")
-                            .setControllerDescriptor(
-                                TezEntityDescriptorProto
-                                    .newBuilder()
-                                    .setClassName("inputinitlizer"))
-                            .build())
-                    .addOutputs(
-                        DAGProtos.RootInputLeafOutputProto
-                            .newBuilder()
-                            .setIODescriptor(
-                                TezEntityDescriptorProto.newBuilder()
-                                    .setClassName("output").build())
-                            .setName("outputx")
-                            .setControllerDescriptor(
-                                TezEntityDescriptorProto
-                                    .newBuilder()
-                                    .setClassName(
-                                        CountingOutputCommitter.class.getName())))
-                    .build()).build();
-    return dag;
-  }
-
-  /*
-   * v1
-   *  |
-   * v2
-   */
-  private DAGPlan createDAGPlanMR() {
-    DAGPlan dag =
-        DAGPlan
-            .newBuilder()
-            .setName("testverteximpl")
-            .addVertex(
-                VertexPlan
-                    .newBuilder()
-                    .setName("vertex1")
-                    .setType(PlanVertexType.NORMAL)
-                    .addTaskLocationHint(
-                        PlanTaskLocationHint.newBuilder().addHost("host1")
-                            .addRack("rack1").build())
-                    .setTaskConfig(
-                        PlanTaskConfiguration.newBuilder().setNumTasks(1)
-                            .setVirtualCores(4).setMemoryMb(1024)
-                            .setJavaOpts("").setTaskModule("x1.y1").build())
-                    .addOutEdgeId("e1")
-                    .addOutputs(
-                        DAGProtos.RootInputLeafOutputProto
-                            .newBuilder()
-                            .setIODescriptor(
-                                TezEntityDescriptorProto.newBuilder()
-                                    .setClassName("output").build())
-                            .setName("outputx")
-                            .setControllerDescriptor(
-                                TezEntityDescriptorProto
-                                    .newBuilder()
-                                    .setClassName(
-                                        CountingOutputCommitter.class.getName())))
-                    .build())
-            .addVertex(
-                VertexPlan
-                    .newBuilder()
-                    .setName("vertex2")
-                    .setType(PlanVertexType.NORMAL)
-                    .setProcessorDescriptor(
-                        TezEntityDescriptorProto.newBuilder().setClassName(
-                            "x2.y2"))
-                    .addTaskLocationHint(
-                        PlanTaskLocationHint.newBuilder().addHost("host2")
-                            .addRack("rack2").build())
-                    .setTaskConfig(
-                        PlanTaskConfiguration.newBuilder().setNumTasks(2)
-                            .setVirtualCores(4).setMemoryMb(1024)
-                            .setJavaOpts("foo").setTaskModule("x2.y2").build())
-                    .addInEdgeId("e1")
-                    .addOutputs(
-                        DAGProtos.RootInputLeafOutputProto
-                            .newBuilder()
-                            .setIODescriptor(
-                                TezEntityDescriptorProto.newBuilder()
-                                    .setClassName("output").build())
-                            .setName("outputx")
-                            .setControllerDescriptor(
-                                TezEntityDescriptorProto
-                                    .newBuilder()
-                                    .setClassName(
-                                        CountingOutputCommitter.class.getName())))
-                    .build()
-
-            )
-            .addEdge(
-                EdgePlan
-                    .newBuilder()
-                    .setEdgeDestination(
-                        TezEntityDescriptorProto.newBuilder().setClassName(
-                            "i2_v1"))
-                    .setInputVertexName("vertex1")
-                    .setEdgeSource(
-                        TezEntityDescriptorProto.newBuilder()
-                            .setClassName("o1"))
-                    .setOutputVertexName("vertex2")
-                    .setDataMovementType(
-                        PlanEdgeDataMovementType.SCATTER_GATHER).setId("e1")
-                    .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                    .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                    .build())
-            .build();
-
-    return dag;
-  }
-
-  class DAGEventHandler implements EventHandler<DAGEvent> {
-    @Override
-    public void handle(DAGEvent event) {
-      dag.handle(event);
-    }
-  }
-
-  class VertexEventHanlder implements EventHandler<VertexEvent> {
-
-    private List<VertexEvent> events = new ArrayList<VertexEvent>();
-
-    @Override
-    public void handle(VertexEvent event) {
-      events.add(event);
-      ((VertexImpl) dag.getVertex(event.getVertexId())).handle(event);
-    }
-
-    public List<VertexEvent> getEvents() {
-      return this.events;
-    }
-  }
-
-  class TaskEventHandler implements EventHandler<TaskEvent> {
-
-    private List<TaskEvent> events = new ArrayList<TaskEvent>();
-
-    @Override
-    public void handle(TaskEvent event) {
-      events.add(event);
-      ((TaskImpl) dag.getVertex(event.getTaskID().getVertexID()).getTask(
-          event.getTaskID())).handle(event);
-    }
-
-    public List<TaskEvent> getEvents() {
-      return events;
-    }
-  }
-
-  class TaskAttemptEventHandler implements EventHandler<TaskAttemptEvent> {
-
-    @Override
-    public void handle(TaskAttemptEvent event) {
-      // TezTaskID taskId = event.getTaskAttemptID().getTaskID();
-      // ((TaskAttemptImpl) vertex1.getTask(taskId).getAttempt(
-      // event.getTaskAttemptID())).handle(event);
-    }
-  }
-
-  private DAGEventHandler dagEventHandler;
-  private VertexEventHanlder vertexEventHandler;
-  private TaskEventHandler taskEventHandler;
-
-  @Before
-  public void setUp() throws IOException {
-
-    dispatcher = new DrainDispatcher();
-    dispatcher.register(DAGAppMasterEventType.class, mock(EventHandler.class));
-    dagEventHandler = new DAGEventHandler();
-    dispatcher.register(DAGEventType.class, dagEventHandler);
-    vertexEventHandler = new VertexEventHanlder();
-    dispatcher.register(VertexEventType.class, vertexEventHandler);
-    taskEventHandler = new TaskEventHandler();
-    dispatcher.register(TaskEventType.class, taskEventHandler);
-    dispatcher.register(TaskAttemptEventType.class,
-        new TaskAttemptEventHandler());
-    dispatcher.init(new Configuration());
-    dispatcher.start();
-
-    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
-
-    DAGPlan dagPlan = createDAGPlan();
-    dag =
-        new DAGImpl(dagId, new Configuration(), dagPlan,
-            dispatcher.getEventHandler(), mock(TaskCommunicatorManagerInterface.class),
-            new Credentials(), new SystemClock(), user,
-            mock(TaskHeartbeatHandler.class), mockAppContext);
-    when(mockAppContext.getCurrentDAG()).thenReturn(dag);
-    ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10));
-    doReturn(clusterInfo).when(mockAppContext).getClusterInfo();
-
-    dag.restoreFromEvent(new DAGInitializedEvent(dag.getID(), 0L, "user", "dagName", null));
-    dag.restoreFromEvent(new DAGStartedEvent(dag.getID(), 0L, "user", "dagName"));
-    LOG.info("finish setUp");
-  }
-
-  /**
-   * vertex1(New) -> StartRecoveryTransition(SUCCEEDED)
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Desired_SUCCEEDED() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
-        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.SUCCEEDED));
-    dispatcher.await();
-    assertEquals(VertexState.SUCCEEDED, vertex1.getState());
-    assertEquals(vertex1.numTasks, vertex1.succeededTaskCount);
-    assertEquals(vertex1.numTasks, vertex1.completedTaskCount);
-    // recover its task
-    assertTaskRecoveredEventSent(vertex1);
-
-    // vertex3 is still in NEW, when the desiredState is
-    // Completed State, each vertex recovery by itself, not depend on its parent
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    assertEquals(VertexState.NEW, vertex3.getState());
-    // no VertexEvent pass to downstream vertex
-    assertEquals(0, vertexEventHandler.getEvents().size());
-
-  }
-
-  @Test(timeout = 5000)
-  public void testRecovery_SetParallelism() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    int oldNumTasks = 10;
-    VertexState recoveredState = vertex1
-        .restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(), "vertex1",
-            initRequestedTime, initedTime, oldNumTasks, "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    recoveredState = vertex1.restoreFromEvent(new VertexParallelismUpdatedEvent(vertex1
-        .getVertexId(), 5, null, null, null, oldNumTasks));
-    assertEquals(5, vertex1.getTotalTasks());
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.SUCCEEDED));
-    dispatcher.await();
-    assertEquals(VertexState.SUCCEEDED, vertex1.getState());
-    assertEquals(vertex1.numTasks, vertex1.succeededTaskCount);
-    assertEquals(vertex1.numTasks, vertex1.completedTaskCount);
-    // recover its task
-    assertTaskRecoveredEventSent(vertex1);
-
-    // vertex3 is still in NEW, when the desiredState is
-    // Completed State, each vertex recovery by itself, not depend on its parent
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    assertEquals(VertexState.NEW, vertex3.getState());
-    // no VertexEvent pass to downstream vertex
-    assertEquals(0, vertexEventHandler.getEvents().size());
-  }
-  
-  @Test(timeout = 5000)
-  public void testRecovery_SetParallelismMultiple() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    int oldNumTasks = 10;
-    VertexState recoveredState = vertex1
-        .restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(), "vertex1",
-            initRequestedTime, initedTime, oldNumTasks, "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    recoveredState = vertex1.restoreFromEvent(new VertexParallelismUpdatedEvent(vertex1
-        .getVertexId(), 5, null, null, null, oldNumTasks));
-    assertEquals(5, vertex1.getTotalTasks());
-    recoveredState = vertex1.restoreFromEvent(new VertexParallelismUpdatedEvent(vertex1
-        .getVertexId(), 7, null, null, null, 5));
-    assertEquals(7, vertex1.getTotalTasks());
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.SUCCEEDED));
-    dispatcher.await();
-    assertEquals(VertexState.SUCCEEDED, vertex1.getState());
-    assertEquals(vertex1.numTasks, vertex1.succeededTaskCount);
-    assertEquals(vertex1.numTasks, vertex1.completedTaskCount);
-    // recover its task
-    assertTaskRecoveredEventSent(vertex1);
-
-    // vertex3 is still in NEW, when the desiredState is
-    // Completed State, each vertex recovery by itself, not depend on its parent
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    assertEquals(VertexState.NEW, vertex3.getState());
-    // no VertexEvent pass to downstream vertex
-    assertEquals(0, vertexEventHandler.getEvents().size());
-
-  }
-
-
-  /**
-   * vertex1(New) -> StartRecoveryTransition(SUCCEEDED)
-   * @throws IOException 
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Desired_SUCCEEDED_OnlySummaryLog() throws IOException {
-    DAGPlan dagPlan = createDAGPlanSingleVertex();
-    dag =
-        new DAGImpl(dagId, new Configuration(), dagPlan,
-            dispatcher.getEventHandler(), mock(TaskCommunicatorManagerInterface.class),
-            new Credentials(), new SystemClock(), user,
-            mock(TaskHeartbeatHandler.class), mockAppContext);
-    when(mockAppContext.getCurrentDAG()).thenReturn(dag);
-    dag.handle(new DAGEvent(dagId, DAGEventType.DAG_INIT));
-
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    VertexFinishedEvent vertexFinishEvent = new VertexFinishedEvent();
-    vertexFinishEvent.fromSummaryProtoStream(SummaryEventProto.newBuilder()
-        .setDagId(dag.getID().toString())
-        .setEventType(HistoryEventType.VERTEX_FINISHED.ordinal())
-        .setTimestamp(100L)
-        .setEventPayload(VertexFinishStateProto.newBuilder()
-            .setNumTasks(2)
-            .setState(VertexState.SUCCEEDED.ordinal())
-            .setVertexId(vertex1.getVertexId().toString()).build().toByteString())
-        .build());
-    VertexState recoveredState = vertex1.restoreFromEvent(vertexFinishEvent);
-    // numTasks is recovered from summary log
-    assertEquals(2, vertex1.numTasks);
-    assertEquals(VertexState.SUCCEEDED, recoveredState);
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.SUCCEEDED));
-    dispatcher.await();
-    assertEquals(VertexState.SUCCEEDED, vertex1.getState());
-    assertEquals(vertex1.numTasks, vertex1.succeededTaskCount);
-    assertEquals(vertex1.numTasks, vertex1.completedTaskCount);
-  }
-
-  /**
-   * vertex1(New) -> StartRecoveryTransition(FAILED)
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Desired_FAILED() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
-        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.FAILED));
-    dispatcher.await();
-    assertEquals(VertexState.FAILED, vertex1.getState());
-    assertEquals(vertex1.numTasks, vertex1.failedTaskCount);
-    assertEquals(0, vertex1.completedTaskCount);
-    // recover its task
-    assertTaskRecoveredEventSent(vertex1);
-
-    // vertex3 is still in NEW, when the desiredState is
-    // Completed State, each vertex recovery by itself, not depend on its parent
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    assertEquals(VertexState.NEW, vertex3.getState());
-    // no VertexEvent pass to downstream vertex
-    assertEquals(0, vertexEventHandler.getEvents().size());
-  }
-
-  /**
-   * vertex1(New) -> StartRecoveryTransition(KILLED)
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Desired_KILLED() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
-        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.KILLED));
-    dispatcher.await();
-    assertEquals(VertexState.KILLED, vertex1.getState());
-    assertEquals(vertex1.numTasks, vertex1.killedTaskCount);
-    assertEquals(0, vertex1.completedTaskCount);
-    // recover its task
-    assertTaskRecoveredEventSent(vertex1);
-
-    // vertex3 is still in NEW, when the desiredState is
-    // Completed State, each vertex recovery by itself, not depend on its parent
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    assertEquals(VertexState.NEW, vertex3.getState());
-    // no VertexEvent pass to downstream vertex
-    assertEquals(0, vertexEventHandler.getEvents().size());
-  }
-
-  /**
-   * vertex1(New) -> StartRecoveryTransition(ERROR)
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Desired_ERROR() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexInitializedEvent(vertex1.getVertexId(),
-        "vertex1", initRequestedTime, initedTime, vertex1.getTotalTasks(), "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.ERROR));
-    dispatcher.await();
-    assertEquals(VertexState.ERROR, vertex1.getState());
-    assertEquals(vertex1.numTasks, vertex1.failedTaskCount);
-    assertEquals(0, vertex1.completedTaskCount);
-    // recover its task
-    assertTaskRecoveredEventSent(vertex1);
-
-    // vertex3 is still in NEW, when the desiredState is
-    // Completed State, each vertex recovery by itself, not depend on its parent
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    assertEquals(VertexState.NEW, vertex3.getState());
-    // no VertexEvent pass to downstream vertex
-    assertEquals(0, vertexEventHandler.getEvents().size());
-  }
-
-  private TezEvent createTezEvent() {
-    return new TezEvent(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.allocate(0)),
-        new EventMetaData(EventProducerConsumerType.INPUT, "vertex1", null,
-            null));
-  }
-
-  /**
-   * vertex1(New) -> restoreFromDataMovementEvent -> StartRecoveryTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_New_Desired_RUNNING() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    VertexState recoveredState =
-        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
-            vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
-    assertEquals(VertexState.NEW, recoveredState);
-    assertEquals(1, vertex1.recoveredEvents.size());
-
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-
-    // InputDataInformationEvent is removed
-    assertEquals(0, vertex1.recoveredEvents.size());
-    // V_INIT and V_START is sent
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-
-  }
-
-  private void assertTaskRecoveredEventSent(VertexImpl vertex) {
-    int sentNum = 0;
-    for (TaskEvent event : taskEventHandler.getEvents()) {
-      if (event.getType() == TaskEventType.T_RECOVER) {
-        TaskEventRecoverTask recoverEvent = (TaskEventRecoverTask)event;
-        if (recoverEvent.getTaskID().getVertexID().equals(vertex.getVertexId())){
-          sentNum++;
-        }
-      }
-    }
-    assertEquals("expect " + vertex.getTotalTasks()
-        + " TaskEventTaskRecover sent for vertex:" + vertex.getVertexId() +
-        "but actuall sent " + sentNum, vertex.getTotalTasks(), sentNum);
-  }
-
-  private void assertOutputCommitters(VertexImpl vertex){
-    assertTrue(vertex.getOutputCommitters() != null);
-    for (OutputCommitter c : vertex.getOutputCommitters().values()) {
-      CountingOutputCommitter committer = (CountingOutputCommitter) c;
-      assertEquals(0, committer.abortCounter);
-      assertEquals(0, committer.commitCounter);
-      assertEquals(1, committer.initCounter);
-      assertEquals(1, committer.setupCounter);
-    }
-  }
-
-  private void restoreFromInitializedEvent(VertexImpl vertex) {
-    long initTimeRequested = 100L;
-    long initedTime = initTimeRequested + 100L;
-    VertexState recoveredState =
-        vertex.restoreFromEvent(new VertexInitializedEvent(vertex
-            .getVertexId(), "vertex1", initTimeRequested, initedTime, vertex.getTotalTasks(),
-            "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    assertEquals(vertex.getTotalTasks(), vertex.getTasks().size());
-    assertEquals(initTimeRequested, vertex.initTimeRequested);
-    assertEquals(initedTime, vertex.initedTime);
-  }
-
-  /**
-   * restoreFromVertexInitializedEvent -> StartRecoveryTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Inited_Desired_RUNNING() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-
-    VertexState recoveredState =
-        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
-            vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
-    assertEquals(VertexState.INITED, recoveredState);
-
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-
-    // InputDataInformationEvent is removed
-    assertEquals(0, vertex1.recoveredEvents.size());
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    // task recovered event is sent
-    assertTaskRecoveredEventSent(vertex1);
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-  }
-
-  /**
-   * restoreFromVertexInitializedEvent -> restoreFromVertexStartedEvent ->
-   * StartRecoveryTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Started_Desired_RUNNING() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-
-    long startTimeRequested = initedTime + 100L;
-    long startedTime = startTimeRequested + 100L;
-    VertexState recoveredState =
-        vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
-            startTimeRequested, startedTime));
-    assertEquals(VertexState.RUNNING, recoveredState);
-    assertEquals(startTimeRequested, vertex1.startTimeRequested);
-    assertEquals(startedTime, vertex1.startedTime);
-
-    recoveredState =
-        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
-            vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
-    assertEquals(VertexState.RUNNING, recoveredState);
-    assertEquals(1, vertex1.recoveredEvents.size());
-
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-
-    // InputDataInformationEvent is removed
-    assertEquals(0, vertex1.recoveredEvents.size());
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    // task recovered event is sent
-    assertTaskRecoveredEventSent(vertex1);
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-  }
-
-  /**
-   * restoreFromVertexInitializedEvent -> restoreFromVertexStartedEvent ->
-   * restoreFromVertexFinishedEvent -> StartRecoveryTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Finished_Desired_RUNNING() {
-    // v1: initFromInitializedEvent
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-
-    // v1: initFromStartedEvent
-    long startRequestedTime = initedTime + 100L;
-    long startTime = startRequestedTime + 100L;
-    VertexState recoveredState =
-        vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
-            startRequestedTime, startTime));
-    assertEquals(VertexState.RUNNING, recoveredState);
-
-    // v1: initFromFinishedEvent
-    long finishTime = startTime + 100L;
-    recoveredState =
-        vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
-            "vertex1", 1, initRequestedTime, initedTime, startRequestedTime,
-            startTime, finishTime, VertexState.SUCCEEDED, "",
-            new TezCounters(), new VertexStats(), null));
-    assertEquals(finishTime, vertex1.finishTime);
-    assertEquals(VertexState.SUCCEEDED, recoveredState);
-    assertEquals(false, vertex1.recoveryCommitInProgress);
-
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-
-    // InputDataInformationEvent is removed
-    assertEquals(0, vertex1.recoveredEvents.size());
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    // task recovered event is sent
-    assertTaskRecoveredEventSent(vertex1);
-
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-  }
-
-  /**
-   * vertex1 (New) -> StartRecoveryTransition <br>
-   * vertex2 (New) -> StartRecoveryTransition <br>
-   * vertex3 (New) -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_RecoveringFromNew() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    assertEquals(1, vertex1.getTasks().size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    VertexState recoveredState =
-        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
-            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
-    assertEquals(VertexState.NEW, recoveredState);
-    assertEquals(1, vertex3.recoveredEvents.size());
-
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-
-    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
-    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex2.getState());
-    // no OutputCommitter for vertex2
-    assertNull(vertex2.getOutputCommitters());
-
-    // v3 go to RUNNING because v1 and v2 both start
-    assertEquals(VertexState.RUNNING, vertex3.getState());
-    assertEquals(2, vertex3.numRecoveredSourceVertices);
-    assertEquals(2, vertex3.numInitedSourceVertices);
-    assertEquals(2, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-    // RootInputDataInformation is removed
-    assertEquals(0, vertex3.recoveredEvents.size());
-
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex3);
-
-  }
-  
-  /**
-   * vertex1 (New) -> StartRecoveryTransition <br>
-   * vertex2 (New) -> RecoveryTransition <br>
-   */
-  @Test
-  public void testMRDAG() {
-    DAGPlan dagPlan = createDAGPlanMR();
-    dag =
-        new DAGImpl(dagId, new Configuration(), dagPlan,
-            dispatcher.getEventHandler(), mock(TaskCommunicatorManagerInterface.class),
-            new Credentials(), new SystemClock(), user,
-            mock(TaskHeartbeatHandler.class), mockAppContext);
-    when(mockAppContext.getCurrentDAG()).thenReturn(dag);
-    dag.handle(new DAGEvent(dagId, DAGEventType.DAG_INIT));
-
-    VertexImpl vertex1 = (VertexImpl)dag.getVertex("vertex1");
-    VertexImpl vertex2 = (VertexImpl)dag.getVertex("vertex2");
-    assertEquals(VertexState.NEW, vertex1.getState());
-    assertEquals(VertexState.NEW, vertex1.getState());
-
-    // vertex1 handle RecoveryEvent at the state of NEW
-    // vertex 2 handle SourceVertexRecoveryEvent at the state of NEW
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    assertEquals(1, vertex1.getTasks().size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-    assertEquals(VertexState.RUNNING, vertex2.getState());
-  }
-
-  @Test(timeout = 5000)
-  public void testRecovery_VertexManagerErrorOnRecovery() {
-    // In order to simulate the behavior that VertexManagerError happens in recovering stage, need to start the recovering from
-    // vertex and disable the the eventhandling of DAG (use mock here).
-    dispatcher = new DrainDispatcher();
-    dispatcher.register(DAGEventType.class, mock(EventHandler.class));
-    vertexEventHandler = new VertexEventHanlder();
-    dispatcher.register(VertexEventType.class, vertexEventHandler);
-    taskEventHandler = new TaskEventHandler();
-    dispatcher.register(TaskEventType.class, taskEventHandler);
-    dispatcher.register(TaskAttemptEventType.class,
-        new TaskAttemptEventHandler());
-    dispatcher.init(new Configuration());
-    dispatcher.start();
-    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
-    DAGPlan dagPlan = createDAGPlan();
-    dag =
-        new DAGImpl(dagId, new Configuration(), dagPlan,
-            dispatcher.getEventHandler(), mock(TaskCommunicatorManagerInterface.class),
-            new Credentials(), new SystemClock(), user,
-            mock(TaskHeartbeatHandler.class), mockAppContext);
-    when(mockAppContext.getCurrentDAG()).thenReturn(dag);
-    ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10));
-    doReturn(clusterInfo).when(mockAppContext).getClusterInfo();
-    dag.restoreFromEvent(new DAGInitializedEvent(dag.getID(), 0L, "user", "dagName", null));
-    dag.restoreFromEvent(new DAGStartedEvent(dag.getID(), 0L, "user", "dagName"));
-    LOG.info("finish setUp");
-
-    /////////////////// Start the recover ////////////////////////
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    assertEquals(vertex1.getTotalTasks(), vertex1.getTasks().size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    VertexState recoveredState =
-        vertex3.restoreFromEvent(new VertexInitializedEvent(vertex3
-            .getVertexId(), "vertex3", initRequestedTime, initedTime, 0, "",
-            null));
-    assertEquals(VertexState.INITED, recoveredState);
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    vertex3.handle(new VertexEventManagerUserCodeError(vertex3.getVertexId(), 
-        new AMUserCodeException(Source.VertexManager, new TezUncheckedException("test"))));
-
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
-    restoreFromInitializedEvent(vertex2);
-    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex2.getState());
-
-    // v3 FAILED due to user code error
-    assertEquals(VertexState.FAILED, vertex3.getState());
-    Assert.assertEquals(VertexTerminationCause.AM_USERCODE_FAILURE, vertex3.getTerminationCause());
-    assertEquals(2, vertex3.numRecoveredSourceVertices);
-  }
-
-
-  /**
-   * vertex1 (New) -> restoreFromInitialized -> StartRecoveryTransition<br>
-   * vertex2 (New) -> restoreFromInitialized -> StartRecoveryTransition<br>
-   * vertex3 (New) -> restoreFromVertexInitedEvent -> RecoverTransition<br>
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_RecoveringFromInited() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    assertEquals(vertex1.getTotalTasks(), vertex1.getTasks().size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    VertexState recoveredState =
-        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
-            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
-    assertEquals(VertexState.NEW, recoveredState);
-    assertEquals(1, vertex3.recoveredEvents.size());
-    recoveredState =
-        vertex3.restoreFromEvent(new VertexInitializedEvent(vertex3
-            .getVertexId(), "vertex3", initRequestedTime, initedTime, 2, "",
-            null));
-    assertEquals(VertexState.INITED, recoveredState);
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-
-    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
-    restoreFromInitializedEvent(vertex2);
-    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex2.getState());
-
-    // v3 go to RUNNING because v1 and v2 both start
-    assertEquals(VertexState.RUNNING, vertex3.getState());
-    assertEquals(2, vertex3.numRecoveredSourceVertices);
-    // numInitedSourceVertices is wrong but doesn't matter because v3 has
-    // already initialized
-    assertEquals(2, vertex3.numInitedSourceVertices);
-    assertEquals(2, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-    // RootInputDataInformation is removed
-    assertEquals(0, vertex3.recoveredEvents.size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex3);
-    // 1 for vertex1, 2 for vertex2, the second 2 for vertex3
-    assertTaskRecoveredEventSent(vertex1);
-    assertTaskRecoveredEventSent(vertex2);
-    assertTaskRecoveredEventSent(vertex3);
-  }
-
-  /**
-   * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * StartRecoveryTransition <br>
-   * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> StartRecoveryTransition <br>
-   * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_RecoveringFromRunning() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    assertEquals(VertexState.RUNNING, recoveredState);
-
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    assertEquals(1, vertex1.getTasks().size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    recoveredState =
-        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
-            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
-    assertEquals(VertexState.NEW, recoveredState);
-    assertEquals(1, vertex3.recoveredEvents.size());
-    recoveredState =
-        vertex3.restoreFromEvent(new VertexInitializedEvent(vertex3
-            .getVertexId(), "vertex3", initRequestedTime, initedTime, vertex3.getTotalTasks(), "",
-            null));
-    assertEquals(VertexState.INITED, recoveredState);
-    recoveredState =
-        vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
-            initRequestedTime + 100L, initRequestedTime + 200L));
-    assertEquals(VertexState.RUNNING, recoveredState);
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-
-    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
-    recoveredState = vertex2.restoreFromEvent(new VertexInitializedEvent(vertex2.getVertexId(),
-        "vertex2", initRequestedTime, initedTime, vertex2.getTotalTasks(), "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    assertEquals(VertexState.RUNNING, recoveredState);
-    
-    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex2.getState());
-
-    // v3 go to RUNNING because v1 and v2 both start
-    assertEquals(VertexState.RUNNING, vertex3.getState());
-    assertEquals(2, vertex3.numRecoveredSourceVertices);
-    assertEquals(2, vertex3.numInitedSourceVertices);
-    assertEquals(2, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-    // RootInputDataInformation is removed
-    assertEquals(0, vertex3.recoveredEvents.size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex3);
-
-    assertTaskRecoveredEventSent(vertex1);
-    assertTaskRecoveredEventSent(vertex2);
-    assertTaskRecoveredEventSent(vertex3);
-  }
-
-  /**
-   * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * restoreFromVertexFinished -> StartRecoveryTransition<br>
-   * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * restoreFromVertexFinished -> StartRecoveryTransition<br>
-   * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_RecoveringFromSUCCEEDED() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    assertEquals(VertexState.RUNNING, recoveredState);
-
-    recoveredState = vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
-        "vertex1", 1, initRequestedTime, initedTime, initRequestedTime + 300L,
-        initRequestedTime + 400L, initRequestedTime + 500L,
-        VertexState.SUCCEEDED, "", new TezCounters(), new VertexStats(), null));
-    assertEquals(VertexState.SUCCEEDED, recoveredState);
-
-    vertex1.handle(new VertexEventRecoverVertex(vertex1.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex1.getState());
-    assertEquals(1, vertex1.getTasks().size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex1);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    recoveredState =
-        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
-            vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
-    assertEquals(VertexState.NEW, recoveredState);
-    assertEquals(1, vertex3.recoveredEvents.size());
-    restoreFromInitializedEvent(vertex3);
-    recoveredState =
-        vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
-            initRequestedTime + 100L, initRequestedTime + 200L));
-    assertEquals(VertexState.RUNNING, recoveredState);
-    // wait for recovery of vertex2
-    assertEquals(VertexState.RECOVERING, vertex3.getState());
-    assertEquals(1, vertex3.numRecoveredSourceVertices);
-    assertEquals(1, vertex3.numInitedSourceVertices);
-    assertEquals(1, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-
-    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
-    recoveredState = vertex2.restoreFromEvent(new VertexInitializedEvent(vertex2.getVertexId(),
-        "vertex2", initRequestedTime, initedTime, vertex2.getTotalTasks(), "", null));
-    assertEquals(VertexState.INITED, recoveredState);
-    recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    assertEquals(VertexState.RUNNING, recoveredState);
-    vertex2.handle(new VertexEventRecoverVertex(vertex2.getVertexId(),
-        VertexState.RUNNING));
-    dispatcher.await();
-    assertEquals(VertexState.RUNNING, vertex2.getState());
-
-    // v3 go to RUNNING because v1 and v2 both start
-    assertEquals(VertexState.RUNNING, vertex3.getState());
-    assertEquals(2, vertex3.numRecoveredSourceVertices);
-    assertEquals(2, vertex3.numInitedSourceVertices);
-    assertEquals(2, vertex3.numStartedSourceVertices);
-    assertEquals(1, vertex3.getDistanceFromRoot());
-    // RootInputDataInformation is removed
-    assertEquals(0, vertex3.recoveredEvents.size());
-    // verify OutputCommitter is initialized
-    assertOutputCommitters(vertex3);
-
-    assertTaskRecoveredEventSent(vertex1);
-    assertTaskRecoveredEventSent(vertex2);
-    assertTaskRecoveredEventSent(vertex3);
-  }
-
-  /**
-   * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * restoreFromVertexFinished (KILLED)
-   * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * restoreFromVertexFinished (KILLED)
-   * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * restoreFromVertexFinished (KILLED)
-  */
-  @Test(timeout = 5000)
-  public void testRecovery_KilledBeforeTaskStarted() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    recoveredState = vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
-        "vertex1", 1, initRequestedTime, initedTime, initRequestedTime + 300L,
-        initRequestedTime + 400L, initRequestedTime + 500L,
-        VertexState.KILLED, "", new TezCounters(), new VertexStats(), null));
-    assertEquals(VertexState.KILLED, recoveredState);
-
-    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
-    restoreFromInitializedEvent(vertex2);
-    recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    recoveredState = vertex2.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
-        "vertex2", 1, initRequestedTime, initedTime, initRequestedTime + 300L,
-        initRequestedTime + 400L, initRequestedTime + 500L,
-        VertexState.KILLED, "", new TezCounters(), new VertexStats(), null));
-    assertEquals(VertexState.KILLED, recoveredState);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    restoreFromInitializedEvent(vertex3);
-    recoveredState = vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    recoveredState = vertex3.restoreFromEvent(new VertexFinishedEvent(vertex3.getVertexId(),
-        "vertex3", 1, initRequestedTime, initedTime, initRequestedTime + 300L,
-        initRequestedTime + 400L, initRequestedTime + 500L,
-        VertexState.KILLED, "", new TezCounters(), new VertexStats(), null));
-    assertEquals(VertexState.KILLED, recoveredState);
-
-    // start the recovering, send RecoverEvent to its root vertices (v1, v2)
-    dag.handle(new DAGEventRecoverEvent(dag.getID(), null));
-    dispatcher.await();
-    // recover v1 to KILLED directly and also its tasks are recovered to KILLED
-    assertEquals(VertexState.KILLED, vertex1.getState());
-    for (Task task : vertex1.tasks.values()) {
-      assertEquals(TaskState.KILLED, task.getState());
-    }
-    // recover v2 to KILLED directly and also its tasks are recovered to KILLED
-    assertEquals(VertexState.KILLED, vertex2.getState());
-    for (Task task : vertex2.tasks.values()) {
-      assertEquals(TaskState.KILLED, task.getState());
-    }
-    // recover v3 to KILLED directly and also its tasks are recovered to KILLED
-    assertEquals(VertexState.KILLED, vertex3.getState());
-    for (Task task : vertex3.tasks.values()) {
-      assertEquals(TaskState.KILLED, task.getState());
-    }
-  }
-
-  /**
-   * vertex1 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * restoreFromVertexFinished (FAILED)
-   * vertex2 (New) -> restoreFromInitialized -> restoreFromVertexStarted
-   * vertex3 (New) -> restoreFromInitialized -> restoreFromVertexStarted ->
-   * restoreFromVertexFinished (FAILED)
-  */
-  @Test(timeout = 5000)
-  public void testRecovery_FailedBeforeTaskStarted() {
-    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
-    restoreFromInitializedEvent(vertex1);
-    VertexState recoveredState = vertex1.restoreFromEvent(new VertexStartedEvent(vertex1.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    recoveredState = vertex1.restoreFromEvent(new VertexFinishedEvent(vertex1.getVertexId(),
-        "vertex1", 1, initRequestedTime, initedTime, initRequestedTime + 300L,
-        initRequestedTime + 400L, initRequestedTime + 500L,
-        VertexState.FAILED, "", new TezCounters(), new VertexStats(), null));
-    assertEquals(VertexState.FAILED, recoveredState);
-
-    VertexImpl vertex2 = (VertexImpl) dag.getVertex("vertex2");
-    restoreFromInitializedEvent(vertex2);
-    recoveredState = vertex2.restoreFromEvent(new VertexStartedEvent(vertex2.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    assertEquals(VertexState.RUNNING, recoveredState);
-
-    VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
-    restoreFromInitializedEvent(vertex3);
-    recoveredState = vertex3.restoreFromEvent(new VertexStartedEvent(vertex3.getVertexId(),
-        initRequestedTime + 100L, initRequestedTime + 200L));
-    recoveredState = vertex3.restoreFromEvent(new VertexFinishedEvent(vertex3.getVertexId(),
-        "vertex3", 1, initRequestedTime, initedTime, initRequestedTime + 300L,
-        initRequestedTime + 400L, initRequestedTime + 500L,
-        VertexState.FAILED, "", new TezCounters(), new VertexStats(), null));
-    assertEquals(VertexState.FAILED, recoveredState);
-
-    // start the recovering from DAG
-    dag.handle(new DAGEventRecoverEvent(dag.getID(), null));
-    dispatcher.await();
-    // recover v1 to KILLED directly and also its tasks are recovered to KILLED
-    assertEquals(VertexState.FAILED, vertex1.getState());
-    for (Task task : vertex1.tasks.values()) {
-      assertEquals(TaskState.FAILED, task.getState());
-    }
-    // recover v2 to KILLED finally due to v1/v3 failed will cause dag failed which result in
-    // dag kill all its vertices
-    assertEquals(VertexState.KILLED, vertex2.getState());
-
-    // recover v3 to KILLED directly and also its tasks are recovered to KILLED
-    assertEquals(VertexState.FAILED, vertex3.getState());
-    for (Task task : vertex3.tasks.values()) {
-      assertEquals(TaskState.FAILED, task.getState());
-    }
-    assertEquals(DAGState.FAILED, dag.getState());
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
index 57a849b..14aed3d 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/events/TestHistoryEventsProtoConversion.java
@@ -61,24 +61,29 @@ import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
 import org.apache.tez.runtime.api.InputSpecUpdate;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
 import org.apache.tez.runtime.api.impl.EventMetaData;
 import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.api.impl.EventType;
 import org.apache.tez.runtime.api.impl.TezEvent;
 import org.junit.Assert;
 import org.junit.Test;
 
+import com.google.common.collect.Collections2;
 import com.google.common.collect.Lists;
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
 import java.util.HashMap;
-import java.util.HashSet;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+
 public class TestHistoryEventsProtoConversion {
 
   private static final Logger LOG = LoggerFactory.getLogger(
@@ -270,10 +275,12 @@ public class TestHistoryEventsProtoConversion {
   }
 
   private void testVertexInitializedEvent() throws Exception {
+    List<TezEvent> initGeneratedEvents = Lists.newArrayList(
+        new TezEvent(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.wrap(new byte[0])), null));
     VertexInitializedEvent event = new VertexInitializedEvent(
         TezVertexID.getInstance(
             TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111),
-        "vertex1", 1000l, 15000l, 100, "procName", null);
+        "vertex1", 1000l, 15000l, 100, "procName", null, initGeneratedEvents);
     VertexInitializedEvent deserializedEvent = (VertexInitializedEvent)
         testProtoConversion(event);
     Assert.assertEquals(event.getVertexID(), deserializedEvent.getVertexID());
@@ -286,6 +293,11 @@ public class TestHistoryEventsProtoConversion {
     Assert.assertEquals(event.getAdditionalInputs(),
         deserializedEvent.getAdditionalInputs());
     Assert.assertNull(deserializedEvent.getProcessorName());
+    Assert.assertEquals(1, event.getInitGeneratedEvents().size());
+    Assert.assertEquals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT,
+        event.getInitGeneratedEvents().get(0).getEventType());
+    Assert.assertEquals(event.getInitGeneratedEvents().size(),
+        deserializedEvent.getInitGeneratedEvents().size());
     logEvents(event, deserializedEvent);
   }
 
@@ -304,85 +316,72 @@ public class TestHistoryEventsProtoConversion {
     logEvents(event, deserializedEvent);
   }
 
-  private void testVertexParallelismUpdatedEvent() throws Exception {
-    {
-      InputSpecUpdate rootInputSpecUpdateBulk = InputSpecUpdate
-          .createAllTaskInputSpecUpdate(2);
-      InputSpecUpdate rootInputSpecUpdatePerTask = InputSpecUpdate
-          .createPerTaskInputSpecUpdate(Lists.newArrayList(1, 2, 3));
-      Map<String, InputSpecUpdate> rootInputSpecUpdates = new HashMap<String, InputSpecUpdate>();
-      rootInputSpecUpdates.put("input1", rootInputSpecUpdateBulk);
-      rootInputSpecUpdates.put("input2", rootInputSpecUpdatePerTask);
-      VertexParallelismUpdatedEvent event =
-          new VertexParallelismUpdatedEvent(
-              TezVertexID.getInstance(
-                  TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111),
-              100, null, null, rootInputSpecUpdates, 1);
-      VertexParallelismUpdatedEvent deserializedEvent = (VertexParallelismUpdatedEvent)
+  private void testVertexReconfigureDoneEvent() throws Exception {
+    VertexLocationHint vertexLocationHint = VertexLocationHint.create(new ArrayList<TaskLocationHint>());
+    InputSpecUpdate rootInputSpecUpdateBulk = InputSpecUpdate
+        .createAllTaskInputSpecUpdate(2);
+    InputSpecUpdate rootInputSpecUpdatePerTask = InputSpecUpdate
+        .createPerTaskInputSpecUpdate(Lists.newArrayList(1, 2, 3));
+    Map<String, InputSpecUpdate> rootInputSpecUpdates = new HashMap<String, InputSpecUpdate>();
+    rootInputSpecUpdates.put("input1", rootInputSpecUpdateBulk);
+    rootInputSpecUpdates.put("input2", rootInputSpecUpdatePerTask);
+    
+    Map<String, EdgeProperty> sourceEdgeManagers
+      = new HashMap<String, EdgeProperty>();
+    // add standard and custom edge
+    sourceEdgeManagers.put("foo", EdgeProperty.create(DataMovementType.SCATTER_GATHER, 
+        DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+        OutputDescriptor.create("Out1"), InputDescriptor.create("in1")));
+    sourceEdgeManagers.put("foo1", EdgeProperty.create(EdgeManagerPluginDescriptor.create("bar1")
+        .setUserPayload(
+            UserPayload.create(ByteBuffer.wrap(new String("payload").getBytes()), 100)), 
+        DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
+        OutputDescriptor.create("Out1"), InputDescriptor.create("in1")));
+
+    VertexConfigurationDoneEvent event =
+        new VertexConfigurationDoneEvent(
+            TezVertexID.getInstance(
+                TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111),
+            100, 2, vertexLocationHint, sourceEdgeManagers, rootInputSpecUpdates, true);
+    VertexConfigurationDoneEvent deserializedEvent = (VertexConfigurationDoneEvent)
           testProtoConversion(event);
-      Assert.assertEquals(event.getVertexID(), deserializedEvent.getVertexID());
-      Assert.assertEquals(event.getNumTasks(), deserializedEvent.getNumTasks());
-      Assert.assertEquals(event.getSourceEdgeProperties(),
-          deserializedEvent.getSourceEdgeProperties());
-      Assert.assertEquals(event.getVertexLocationHint(),
-          deserializedEvent.getVertexLocationHint());
-      Assert.assertEquals(event.getRootInputSpecUpdates().size(), deserializedEvent
-          .getRootInputSpecUpdates().size());
-      InputSpecUpdate deserializedBulk = deserializedEvent.getRootInputSpecUpdates().get("input1");
-      InputSpecUpdate deserializedPerTask = deserializedEvent.getRootInputSpecUpdates().get("input2");
-      Assert.assertEquals(rootInputSpecUpdateBulk.isForAllWorkUnits(),
-          deserializedBulk.isForAllWorkUnits());
-      Assert.assertEquals(rootInputSpecUpdateBulk.getAllNumPhysicalInputs(),
-          deserializedBulk.getAllNumPhysicalInputs());
-      Assert.assertEquals(rootInputSpecUpdatePerTask.isForAllWorkUnits(),
-          deserializedPerTask.isForAllWorkUnits());
-      Assert.assertEquals(rootInputSpecUpdatePerTask.getAllNumPhysicalInputs(),
-          deserializedPerTask.getAllNumPhysicalInputs());
-      logEvents(event, deserializedEvent);
-    }
-    {
-      Map<String, EdgeProperty> sourceEdgeManagers
-          = new LinkedHashMap<String, EdgeProperty>();
-      // add standard and custom edge
-      sourceEdgeManagers.put("foo", EdgeProperty.create(DataMovementType.SCATTER_GATHER, 
-          DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-          OutputDescriptor.create("Out1"), InputDescriptor.create("in1")));
-      sourceEdgeManagers.put("foo1", EdgeProperty.create(EdgeManagerPluginDescriptor.create("bar1")
-          .setUserPayload(
-              UserPayload.create(ByteBuffer.wrap(new String("payload").getBytes()), 100)), 
-          DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL, 
-          OutputDescriptor.create("Out1"), InputDescriptor.create("in1")));
-      VertexParallelismUpdatedEvent event =
-          new VertexParallelismUpdatedEvent(
-              TezVertexID.getInstance(
-                  TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111),
-              100, VertexLocationHint.create(Arrays.asList(TaskLocationHint.createTaskLocationHint(
-              new HashSet<String>(Arrays.asList("h1")),
-              new HashSet<String>(Arrays.asList("r1"))))),
-              sourceEdgeManagers, null, 1);
-
-      VertexParallelismUpdatedEvent deserializedEvent = 
-          (VertexParallelismUpdatedEvent) testProtoConversion(event);
-      Assert.assertEquals(event.getVertexID(), deserializedEvent.getVertexID());
-      Assert.assertEquals(event.getNumTasks(), deserializedEvent.getNumTasks());
-      Assert.assertEquals(event.getSourceEdgeProperties().size(), deserializedEvent
-          .getSourceEdgeProperties().size());
-      Assert.assertEquals(event.getSourceEdgeProperties().get("foo").getDataMovementType(),
-          deserializedEvent.getSourceEdgeProperties().get("foo").getDataMovementType());
-      Assert.assertNull(deserializedEvent.getSourceEdgeProperties().get("foo")
-          .getEdgeManagerDescriptor());
-      Assert.assertEquals(event.getSourceEdgeProperties().get("foo1").getDataMovementType(),
-          deserializedEvent.getSourceEdgeProperties().get("foo1").getDataMovementType());
-      Assert.assertEquals(event.getSourceEdgeProperties().get("foo1").getEdgeManagerDescriptor()
-          .getUserPayload().getVersion(), deserializedEvent.getSourceEdgeProperties().get("foo1")
-          .getEdgeManagerDescriptor().getUserPayload().getVersion());
-      Assert.assertArrayEquals(event.getSourceEdgeProperties().get("foo1")
-          .getEdgeManagerDescriptor().getUserPayload().deepCopyAsArray(), deserializedEvent
-          .getSourceEdgeProperties().get("foo1").getEdgeManagerDescriptor().getUserPayload()
-          .deepCopyAsArray());
-      Assert.assertEquals(event.getVertexLocationHint(), deserializedEvent.getVertexLocationHint());
-      logEvents(event, deserializedEvent);
-    }
+    Assert.assertEquals(event.getVertexID(), deserializedEvent.getVertexID());
+    Assert.assertEquals(event.getNumTasks(), deserializedEvent.getNumTasks());
+    Assert.assertEquals(event.isSetParallelismCalled(), deserializedEvent.isSetParallelismCalled());
+    // vertexLocationHint
+    Assert.assertEquals(event.getVertexLocationHint(),
+        deserializedEvent.getVertexLocationHint());
+    // rootInputSpec
+    Assert.assertEquals(event.getRootInputSpecUpdates().size(), deserializedEvent
+        .getRootInputSpecUpdates().size());
+    InputSpecUpdate deserializedBulk = deserializedEvent.getRootInputSpecUpdates().get("input1");
+    InputSpecUpdate deserializedPerTask = deserializedEvent.getRootInputSpecUpdates().get("input2");
+    Assert.assertEquals(rootInputSpecUpdateBulk.isForAllWorkUnits(),
+        deserializedBulk.isForAllWorkUnits());
+    Assert.assertEquals(rootInputSpecUpdateBulk.getAllNumPhysicalInputs(),
+        deserializedBulk.getAllNumPhysicalInputs());
+    Assert.assertEquals(rootInputSpecUpdatePerTask.isForAllWorkUnits(),
+        deserializedPerTask.isForAllWorkUnits());
+    Assert.assertEquals(rootInputSpecUpdatePerTask.getAllNumPhysicalInputs(),
+        deserializedPerTask.getAllNumPhysicalInputs());
+    // sourceEdgeManager
+    Assert.assertEquals(event.getSourceEdgeProperties().size(), deserializedEvent
+        .getSourceEdgeProperties().size());
+    Assert.assertEquals(event.getSourceEdgeProperties().get("foo").getDataMovementType(),
+        deserializedEvent.getSourceEdgeProperties().get("foo").getDataMovementType());
+    Assert.assertNull(deserializedEvent.getSourceEdgeProperties().get("foo")
+        .getEdgeManagerDescriptor());
+    Assert.assertEquals(event.getSourceEdgeProperties().get("foo1").getDataMovementType(),
+        deserializedEvent.getSourceEdgeProperties().get("foo1").getDataMovementType());
+    Assert.assertEquals(event.getSourceEdgeProperties().get("foo1").getEdgeManagerDescriptor()
+        .getUserPayload().getVersion(), deserializedEvent.getSourceEdgeProperties().get("foo1")
+        .getEdgeManagerDescriptor().getUserPayload().getVersion());
+    Assert.assertArrayEquals(event.getSourceEdgeProperties().get("foo1")
+        .getEdgeManagerDescriptor().getUserPayload().deepCopyAsArray(), deserializedEvent
+        .getSourceEdgeProperties().get("foo1").getEdgeManagerDescriptor().getUserPayload()
+        .deepCopyAsArray());
+
+    logEvents(event, deserializedEvent);
   }
 
   private void testVertexFinishedEvent() throws Exception {
@@ -500,7 +499,7 @@ public class TestHistoryEventsProtoConversion {
           TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance(
               TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 1),
           "vertex1", 10001l, 1000434444l, TaskAttemptState.FAILED,
-          null, null, null, null, 2048,
+          null, null, null, null, null, 2048,
           TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance(
               TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 0), 1024);
       TaskAttemptFinishedEvent deserializedEvent = (TaskAttemptFinishedEvent)
@@ -536,7 +535,8 @@ public class TestHistoryEventsProtoConversion {
           TezTaskAttemptID.getInstance(TezTaskID.getInstance(TezVertexID.getInstance(
               TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 111), 1), 1),
           "vertex1", 10001l, 1000434444l, TaskAttemptState.FAILED,
-          TaskAttemptTerminationCause.APPLICATION_ERROR, "diagnose", new TezCounters(), events, 0, null, 0);
+          TaskAttemptTerminationCause.APPLICATION_ERROR, "diagnose", new TezCounters(), events,
+          null, 0, null, 0);
       TaskAttemptFinishedEvent deserializedEvent = (TaskAttemptFinishedEvent)
           testProtoConversion(event);
       Assert.assertEquals(event.getTaskAttemptID(),
@@ -594,35 +594,6 @@ public class TestHistoryEventsProtoConversion {
     logEvents(event, deserializedEvent);
   }
 
-  private void testVertexDataMovementEventsGeneratedEvent() throws Exception {
-    VertexRecoverableEventsGeneratedEvent event;
-    try {
-      event = new VertexRecoverableEventsGeneratedEvent(
-          TezVertexID.getInstance(
-              TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 1), null);
-      fail("Invalid creation should have errored out");
-    } catch (RuntimeException e) {
-      // Expected
-    }
-    long eventTime = 1024;
-    List<TezEvent> events =
-        Arrays.asList(new TezEvent(DataMovementEvent.create(1, null),
-            new EventMetaData(EventProducerConsumerType.SYSTEM, "foo", "bar", null), eventTime));
-    event = new VertexRecoverableEventsGeneratedEvent(
-            TezVertexID.getInstance(
-                TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 1), events);
-    VertexRecoverableEventsGeneratedEvent deserializedEvent =
-        (VertexRecoverableEventsGeneratedEvent) testProtoConversion(event);
-    Assert.assertEquals(event.getVertexID(), deserializedEvent.getVertexID());
-    Assert.assertEquals(1,
-        deserializedEvent.getTezEvents().size());
-    Assert.assertEquals(event.getTezEvents().get(0).getEventType(),
-        deserializedEvent.getTezEvents().get(0).getEventType());
-    Assert.assertEquals(event.getTezEvents().get(0).getEventReceivedTime(),
-        deserializedEvent.getTezEvents().get(0).getEventReceivedTime());
-    logEvents(event, deserializedEvent);
-  }
-
   private void testDAGCommitStartedEvent() throws Exception {
     DAGCommitStartedEvent event = new DAGCommitStartedEvent(
         TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 100l);
@@ -643,15 +614,21 @@ public class TestHistoryEventsProtoConversion {
   }
 
   private void testVertexGroupCommitStartedEvent() throws Exception {
+    TezVertexID vertexId1 = TezVertexID.getInstance(
+        TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 0), 1);
+    TezVertexID vertexId2 = TezVertexID.getInstance(
+        TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 0), 2);
+    Collection<TezVertexID> vertexIds = Lists.newArrayList(vertexId1, vertexId2);
     VertexGroupCommitStartedEvent event = new VertexGroupCommitStartedEvent(
         TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1),
-        "fooGroup", 1000344l);
+        "fooGroup", vertexIds, 1000344l);
     {
       VertexGroupCommitStartedEvent deserializedEvent =
           (VertexGroupCommitStartedEvent) testProtoConversion(event);
       Assert.assertEquals(event.getDagID(), deserializedEvent.getDagID());
       Assert.assertEquals(event.getVertexGroupName(),
           deserializedEvent.getVertexGroupName());
+      Assert.assertEquals(event.getVertexIds(), vertexIds);
       logEvents(event, deserializedEvent);
     }
     {
@@ -664,15 +641,21 @@ public class TestHistoryEventsProtoConversion {
   }
 
   private void testVertexGroupCommitFinishedEvent() throws Exception {
+    TezVertexID vertexId1 = TezVertexID.getInstance(
+        TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 0), 1);
+    TezVertexID vertexId2 = TezVertexID.getInstance(
+        TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 0), 2);
+    Collection<TezVertexID> vertexIds = Lists.newArrayList(vertexId1, vertexId2);
     VertexGroupCommitFinishedEvent event = new VertexGroupCommitFinishedEvent(
         TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1),
-        "fooGroup", 1000344l);
+        "fooGroup", vertexIds, 1000344l);
     {
       VertexGroupCommitFinishedEvent deserializedEvent =
           (VertexGroupCommitFinishedEvent) testProtoConversion(event);
       Assert.assertEquals(event.getDagID(), deserializedEvent.getDagID());
       Assert.assertEquals(event.getVertexGroupName(),
           deserializedEvent.getVertexGroupName());
+      Assert.assertEquals(event.getVertexIds(), vertexIds);
       logEvents(event, deserializedEvent);
     }
     {
@@ -685,7 +668,7 @@ public class TestHistoryEventsProtoConversion {
   }
 
 
-  @Test(timeout = 5000)
+  @Test//(timeout = 5000)
   public void testDefaultProtoConversion() throws Exception {
     for (HistoryEventType eventType : HistoryEventType.values()) {
       switch (eventType) {
@@ -716,8 +699,8 @@ public class TestHistoryEventsProtoConversion {
         case VERTEX_STARTED:
           testVertexStartedEvent();
           break;
-        case VERTEX_PARALLELISM_UPDATED:
-          testVertexParallelismUpdatedEvent();
+        case VERTEX_CONFIGURE_DONE:
+          testVertexReconfigureDoneEvent();
           break;
         case VERTEX_FINISHED:
           testVertexFinishedEvent();
@@ -740,9 +723,6 @@ public class TestHistoryEventsProtoConversion {
         case CONTAINER_STOPPED:
           testContainerStoppedEvent();
           break;
-        case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-          testVertexDataMovementEventsGeneratedEvent();
-          break;
         case DAG_COMMIT_STARTED:
           testDAGCommitStartedEvent();
           break;

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
index bcc3859..606fb85 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/logging/impl/TestHistoryEventJsonConversion.java
@@ -59,12 +59,11 @@ import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
 import org.apache.tez.dag.history.events.TaskFinishedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
 import org.apache.tez.dag.history.events.VertexFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
 import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
 import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.history.utils.DAGUtils;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
@@ -139,13 +138,13 @@ public class TestHistoryEventJsonConversion {
           break;
         case VERTEX_INITIALIZED:
           event = new VertexInitializedEvent(tezVertexID, "v1", random.nextInt(), random.nextInt(),
-              random.nextInt(), "proc", null);
+              random.nextInt(), "proc", null, null);
           break;
         case VERTEX_STARTED:
           event = new VertexStartedEvent(tezVertexID, random.nextInt(), random.nextInt());
           break;
-        case VERTEX_PARALLELISM_UPDATED:
-          event = new VertexParallelismUpdatedEvent(tezVertexID, 1, null, null, null, 10);
+        case VERTEX_CONFIGURE_DONE:
+          event = new VertexConfigurationDoneEvent(tezVertexID, 0L, 1, null, null, null, true);
           break;
         case VERTEX_FINISHED:
           event = new VertexFinishedEvent(tezVertexID, "v1", 1, random.nextInt(), random.nextInt(),
@@ -165,7 +164,8 @@ public class TestHistoryEventJsonConversion {
           break;
         case TASK_ATTEMPT_FINISHED:
           event = new TaskAttemptFinishedEvent(tezTaskAttemptID, "v1", random.nextInt(),
-              random.nextInt(), TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT, null, null, null, 0, null, 0);
+              random.nextInt(), TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT,
+              null, null, null, null, 0, null, 0);
           break;
         case CONTAINER_LAUNCHED:
           event = new ContainerLaunchedEvent(containerId, random.nextInt(),
@@ -174,9 +174,6 @@ public class TestHistoryEventJsonConversion {
         case CONTAINER_STOPPED:
           event = new ContainerStoppedEvent(containerId, random.nextInt(), -1, applicationAttemptId);
           break;
-        case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-          event = new VertexRecoverableEventsGeneratedEvent();
-          break;
         case DAG_COMMIT_STARTED:
           event = new DAGCommitStartedEvent();
           break;
@@ -207,7 +204,7 @@ public class TestHistoryEventJsonConversion {
   }
 
   @Test(timeout = 5000)
-  public void testConvertVertexParallelismUpdatedEvent() throws JSONException {
+  public void testConvertVertexReconfigureDoneEvent() throws JSONException {
     TezVertexID vId = TezVertexID.getInstance(
         TezDAGID.getInstance(
             ApplicationId.newInstance(1l, 1), 1), 1);
@@ -217,8 +214,8 @@ public class TestHistoryEventJsonConversion {
     edgeMgrs.put("a", EdgeProperty.create(EdgeManagerPluginDescriptor.create("a.class")
         .setHistoryText("text"), DataSourceType.PERSISTED, SchedulingType.SEQUENTIAL,
         OutputDescriptor.create("Out"), InputDescriptor.create("In")));
-    VertexParallelismUpdatedEvent event = new VertexParallelismUpdatedEvent(vId, 1, null,
-        edgeMgrs, null, 10);
+    VertexConfigurationDoneEvent event = new VertexConfigurationDoneEvent(vId, 0L, 1, null,
+        edgeMgrs, null, true);
 
     JSONObject jsonObject = HistoryEventJsonConversion.convertToJson(event);
     Assert.assertNotNull(jsonObject);
@@ -229,12 +226,11 @@ public class TestHistoryEventJsonConversion {
     Assert.assertEquals(1, events.length());
 
     JSONObject evt = events.getJSONObject(0);
-    Assert.assertEquals(HistoryEventType.VERTEX_PARALLELISM_UPDATED.name(),
+    Assert.assertEquals(HistoryEventType.VERTEX_CONFIGURE_DONE.name(),
         evt.getString(ATSConstants.EVENT_TYPE));
 
     JSONObject evtInfo = evt.getJSONObject(ATSConstants.EVENT_INFO);
     Assert.assertEquals(1, evtInfo.getInt(ATSConstants.NUM_TASKS));
-    Assert.assertEquals(10, evtInfo.getInt(ATSConstants.OLD_NUM_TASKS));
     Assert.assertNotNull(evtInfo.getJSONObject(ATSConstants.UPDATED_EDGE_MANAGERS));
 
     JSONObject updatedEdgeMgrs = evtInfo.getJSONObject(ATSConstants.UPDATED_EDGE_MANAGERS);
@@ -247,9 +243,6 @@ public class TestHistoryEventJsonConversion {
     Assert.assertEquals("In", updatedEdgeMgr.getString(DAGUtils.EDGE_DESTINATION_CLASS_KEY));
     Assert.assertEquals("a.class", updatedEdgeMgr.getString(DAGUtils.EDGE_MANAGER_CLASS_KEY));
 
-    JSONObject otherInfo = jsonObject.getJSONObject(ATSConstants.OTHER_INFO);
-    Assert.assertEquals(1, otherInfo.getInt(ATSConstants.NUM_TASKS));
-
   }
 
 

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java
index 5922100..281eaa9 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/TezExampleBase.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.io.PrintStream;
 import java.util.Set;
 
+import com.google.common.annotations.VisibleForTesting;
 import com.google.common.collect.Sets;
 
 import org.apache.commons.cli.Options;
@@ -32,6 +33,7 @@ import org.apache.tez.client.CallerContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
 import org.apache.hadoop.security.UserGroupInformation;
@@ -263,4 +265,14 @@ public abstract class TezExampleBase extends Configured implements Tool {
    */
   protected abstract int runJob(String[] args, TezConfiguration tezConf,
                                 TezClient tezClient) throws Exception;
+  
+  @Private
+  @VisibleForTesting
+  public ApplicationId getAppId() {
+    if (tezClientInternal == null) {
+      LOG.warn("TezClient is not initialized, return null for AppId");
+      return null;
+    }
+    return tezClientInternal.getAppMasterApplicationId();
+  }
 }


[4/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
index 409c506..803edf7 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
@@ -15,602 +15,1149 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-
 package org.apache.tez.dag.app.dag.impl;
 
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
 
-import java.net.URL;
+import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.util.ArrayList;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.Callable;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.security.UserGroupInformation;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.event.Event;
+import org.apache.hadoop.yarn.event.DrainDispatcher;
 import org.apache.hadoop.yarn.event.EventHandler;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.tez.common.TezAbstractEvent;
-import org.apache.tez.common.counters.TezCounters;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.tez.common.MockDNSToSwitchMapping;
+import org.apache.tez.common.security.ACLManager;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.client.VertexStatus.State;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.dag.api.oldrecords.TaskState;
+import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.api.records.DAGProtos.EdgePlan;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataMovementType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeDataSourceType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanEdgeSchedulingType;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskConfiguration;
+import org.apache.tez.dag.api.records.DAGProtos.PlanTaskLocationHint;
+import org.apache.tez.dag.api.records.DAGProtos.PlanVertexType;
+import org.apache.tez.dag.api.records.DAGProtos.RootInputLeafOutputProto;
+import org.apache.tez.dag.api.records.DAGProtos.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ClusterInfo;
+import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData;
+import org.apache.tez.dag.app.RecoveryParser.TaskAttemptRecoveryData;
+import org.apache.tez.dag.app.RecoveryParser.TaskRecoveryData;
+import org.apache.tez.dag.app.RecoveryParser.VertexRecoveryData;
 import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAGState;
-import org.apache.tez.dag.app.dag.DAGTerminationCause;
+import org.apache.tez.dag.app.dag.Task;
+import org.apache.tez.dag.app.dag.TaskAttempt;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.app.dag.TaskStateInternal;
+import org.apache.tez.dag.app.dag.TestStateChangeNotifier.StateChangeNotifierForTest;
+import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.VertexState;
+import org.apache.tez.dag.app.dag.event.CallableEvent;
+import org.apache.tez.dag.app.dag.event.CallableEventType;
 import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDAGFinished;
+import org.apache.tez.dag.app.dag.event.DAGAppMasterEventType;
 import org.apache.tez.dag.app.dag.event.DAGEvent;
 import org.apache.tez.dag.app.dag.event.DAGEventRecoverEvent;
 import org.apache.tez.dag.app.dag.event.DAGEventType;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventSchedule;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
+import org.apache.tez.dag.app.dag.event.TaskEvent;
+import org.apache.tez.dag.app.dag.event.TaskEventScheduleTask;
+import org.apache.tez.dag.app.dag.event.TaskEventType;
 import org.apache.tez.dag.app.dag.event.VertexEvent;
-import org.apache.tez.dag.app.dag.event.VertexEventRecoverVertex;
-import org.apache.tez.dag.history.events.DAGCommitStartedEvent;
-import org.apache.tez.dag.history.events.DAGFinishedEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
+import org.apache.tez.dag.app.dag.event.VertexEventType;
+import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter;
+import org.apache.tez.dag.app.dag.impl.TestVertexImpl.EventHandlingRootInputInitializer;
+import org.apache.tez.dag.app.rm.AMSchedulerEvent;
+import org.apache.tez.dag.app.rm.AMSchedulerEventType;
+import org.apache.tez.dag.app.rm.TaskSchedulerManager;
+import org.apache.tez.dag.history.DAGHistoryEvent;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventHandler;
+import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.events.DAGInitializedEvent;
-import org.apache.tez.dag.history.events.DAGKillRequestEvent;
+import org.apache.tez.dag.history.events.DAGRecoveredEvent;
 import org.apache.tez.dag.history.events.DAGStartedEvent;
-import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
-import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.history.events.TaskFinishedEvent;
+import org.apache.tez.dag.history.events.TaskStartedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
+import org.apache.tez.dag.history.events.VertexStartedEvent;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.InputInitializer;
+import org.apache.tez.runtime.api.InputInitializerContext;
+import org.apache.tez.runtime.api.InputSpecUpdate;
+import org.apache.tez.runtime.api.OutputCommitter;
+import org.apache.tez.runtime.api.OutputCommitterContext;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+import org.apache.tez.runtime.api.impl.TaskSpec;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.library.common.InputIdentifier;
+import org.junit.After;
+import org.junit.Assert;
 import org.junit.Before;
+import org.junit.BeforeClass;
 import org.junit.Test;
-import org.mockito.ArgumentCaptor;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
 
-@SuppressWarnings({ "unchecked", "rawtypes" })
 public class TestDAGRecovery {
 
+  private static final Logger LOG = LoggerFactory.getLogger(TestDAGImpl.class);
+  private static Configuration conf;
+  private DrainDispatcher dispatcher;
+  private ListeningExecutorService execService;
+  private Credentials fsTokens;
+  private AppContext appContext;
+  private ACLManager aclManager;
+  private ApplicationAttemptId appAttemptId;
+  private TaskEventDispatcher taskEventDispatcher;
+  private VertexEventDispatcher vertexEventDispatcher;
+  private DagEventDispatcher dagEventDispatcher;
+  private TaskCommunicatorManagerInterface taskCommunicatorManagerInterface;
+  private TaskHeartbeatHandler thh;
+  private Clock clock = new SystemClock();
+  private DAGFinishEventHandler dagFinishEventHandler;
+  private DAGPlan dagPlan;
   private DAGImpl dag;
-  private EventHandler mockEventHandler;
+  private TezDAGID dagId;
+  private UserGroupInformation ugi;
+  private MockHistoryEventHandler historyEventHandler;
+  private TaskAttemptEventDispatcher taskAttemptEventDispatcher;
+  private ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,
+      10));
+  private DAGRecoveryData dagRecoveryData = mock(DAGRecoveryData.class);
+
+  private TezVertexID v1Id;   // first vertex
+  private TezTaskID t1v1Id;     // first task of v1
+  private TezTaskAttemptID ta1t1v1Id;   // first task attempt of the first task of v1
+  private TezVertexID v2Id;
+  private TezTaskID t1v2Id;
+  private TezTaskAttemptID ta1t1v2Id;
+
+  ////////////////////////
+  private Random rand = new Random();
+  private long dagInitedTime = System.currentTimeMillis() + rand.nextInt(100);
+  private long dagStartedTime = dagInitedTime + rand.nextInt(100);
+  private long v1InitedTime = dagStartedTime + rand.nextInt(100);
+  private long v1StartedTime = v1InitedTime + rand.nextInt(100);
+  private int v1NumTask = 10;
+  private long t1StartedTime = v1StartedTime + rand.nextInt(100);
+  private long t1FinishedTime = t1StartedTime + rand.nextInt(100);
+  private long ta1LaunchTime = t1StartedTime + rand.nextInt(100);
+  private long ta1FinishedTime = ta1LaunchTime + rand.nextInt(100);
+  
+  private class DagEventDispatcher implements EventHandler<DAGEvent> {
+    @Override
+    public void handle(DAGEvent event) {
+      dag.handle(event);
+    }
+  }
 
-  private String user = "root";
-  private String dagName = "dag1";
+  private class TaskEventDispatcher implements EventHandler<TaskEvent> {
+    @SuppressWarnings("unchecked")
+    @Override
+    public void handle(TaskEvent event) {
+      TaskImpl task = (TaskImpl) dag.getVertex(event.getTaskID().getVertexID())
+          .getTask(event.getTaskID());
+      task.handle(event);
+    }
+  }
 
-  private AppContext mockAppContext;
-  private ApplicationId appId = ApplicationId.newInstance(
-      System.currentTimeMillis(), 1);
-  private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
-  private long initTime = 100L;
-  private long startTime = initTime + 200L;
-  private long commitStartTime = startTime + 200L;
-  private long finishTime = commitStartTime + 200L;
-  private TezCounters tezCounters = new TezCounters();
+  @SuppressWarnings("unchecked")
+  private class TaskAttemptEventDispatcher implements
+      EventHandler<TaskAttemptEvent> {
+    @Override
+    public void handle(TaskAttemptEvent event) {
+      Vertex vertex = dag.getVertex(event.getTaskAttemptID().getTaskID()
+          .getVertexID());
+      Task task = vertex.getTask(event.getTaskAttemptID().getTaskID());
+      TaskAttempt ta = task.getAttempt(event.getTaskAttemptID());
+      ((EventHandler<TaskAttemptEvent>) ta).handle(event);
+    }
+  }
 
-  @Before
-  public void setUp() {
-    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
-    when(mockAppContext.getCurrentDAG().getDagUGI()).thenReturn(null);
-    ClusterInfo clusterInfo = new ClusterInfo(Resource.newInstance(8192,10));
-    doReturn(clusterInfo).when(mockAppContext).getClusterInfo();
-    mockEventHandler = mock(EventHandler.class);
-    tezCounters.findCounter("grp_1", "counter_1").increment(1);
-
-    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
-    dag =
-        new DAGImpl(dagId, new Configuration(), dagPlan, mockEventHandler,
-            mock(TaskCommunicatorManagerInterface.class), new Credentials(),
-            new SystemClock(), user, mock(TaskHeartbeatHandler.class),
-            mockAppContext);
-  }
-
-  private void assertNewState() {
-    assertEquals(0, dag.getVertices().size());
-    assertEquals(0, dag.edges.size());
-    assertNull(dag.dagScheduler);
-    assertFalse(dag.recoveryCommitInProgress);
-    assertEquals(0, dag.recoveredGroupCommits.size());
-  }
-
-  private void restoreFromDAGInitializedEvent() {
-    DAGState recoveredState =
-        dag.restoreFromEvent(new DAGInitializedEvent(dagId, initTime, user,
-            dagName, null));
-    assertEquals(DAGState.INITED, recoveredState);
-    assertEquals(initTime, dag.initTime);
-    assertEquals(6, dag.getVertices().size());
-    assertEquals(6, dag.edges.size());
-    assertNotNull(dag.dagScheduler);
-  }
-
-  private void restoreFromDAGStartedEvent() {
-    DAGState recoveredState =
-        dag.restoreFromEvent(new DAGStartedEvent(dagId, startTime, user,
-            dagName));
-    assertEquals(startTime, dag.startTime);
-    assertEquals(DAGState.RUNNING, recoveredState);
-  }
-
-  private void restoreFromDAGCommitStartedEvent() {
-    DAGState recoveredState =
-        dag.restoreFromEvent(new DAGCommitStartedEvent(dagId, commitStartTime));
-    assertTrue(dag.recoveryCommitInProgress);
-    assertEquals(DAGState.RUNNING, recoveredState);
-  }
-
-  private void restoreFromVertexGroupCommitStartedEvent() {
-    DAGState recoveredState =
-        dag.restoreFromEvent(new VertexGroupCommitStartedEvent(dagId, "g1",
-            commitStartTime));
-    assertEquals(1, dag.recoveredGroupCommits.size());
-    assertFalse(dag.recoveredGroupCommits.get("g1").booleanValue());
-    assertEquals(DAGState.RUNNING, recoveredState);
-  }
-
-  private void restoreFromVertexGroupCommitFinishedEvent() {
-    DAGState recoveredState =
-        dag.restoreFromEvent(new VertexGroupCommitFinishedEvent(dagId, "g1",
-            commitStartTime + 100L));
-    assertEquals(1, dag.recoveredGroupCommits.size());
-    assertTrue(dag.recoveredGroupCommits.get("g1").booleanValue());
-    assertEquals(DAGState.RUNNING, recoveredState);
-  }
-
-  private void restoreFromDAGFinishedEvent(DAGState finalState) {
-    DAGState recoveredState =
-        dag.restoreFromEvent(new DAGFinishedEvent(dagId, startTime, finishTime,
-            finalState, "", tezCounters, user, dagName, null, null));
-    assertEquals(finishTime, dag.finishTime);
-    assertFalse(dag.recoveryCommitInProgress);
-    assertEquals(finalState, recoveredState);
-    assertEquals(tezCounters, dag.fullCounters);
-  }
-
-  private void restoreFromDAGKillRequestEvent() {
-    dag.restoreFromEvent(new DAGKillRequestEvent(dag.getID(), 0L, false));
+  private class VertexEventDispatcher implements EventHandler<VertexEvent> {
+
+    @SuppressWarnings("unchecked")
+    @Override
+    public void handle(VertexEvent event) {
+      VertexImpl vertex = (VertexImpl) dag.getVertex(event.getVertexId());
+      vertex.handle(event);
+    }
   }
 
-  /**
-   * New -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_FromNew() {
-    assertNewState();
+  private class DAGFinishEventHandler implements
+      EventHandler<DAGAppMasterEventDAGFinished> {
+    public int dagFinishEvents = 0;
 
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    @Override
+    public void handle(DAGAppMasterEventDAGFinished event) {
+      ++dagFinishEvents;
+    }
+  }
 
-    ArgumentCaptor<DAGEvent> eventCaptor =
-        ArgumentCaptor.forClass(DAGEvent.class);
-    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
-    List<DAGEvent> events = eventCaptor.getAllValues();
-    assertEquals(2, events.size());
-    assertEquals(DAGEventType.DAG_INIT, events.get(0).getType());
-    assertEquals(DAGEventType.DAG_START, events.get(1).getType());
+  private class AMSchedulerEventDispatcher implements EventHandler<AMSchedulerEvent> {
+    @Override
+    public void handle(AMSchedulerEvent event) {
+    }
   }
+  
+  private static class MockHistoryEventHandler extends HistoryEventHandler {
 
-  /**
-   * New -> restoreFromDAGKillRequested -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_FromNewToKilled() {
-    restoreFromDAGKillRequestEvent();
-    assertNewState();
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.KILLED, dag.getState());
-    assertEquals(DAGTerminationCause.DAG_KILL, dag.getTerminationCause());
+    private List<HistoryEvent> historyEvents = new ArrayList<HistoryEvent>();
+
+    public MockHistoryEventHandler(AppContext context) {
+      super(context);
+    }
+
+    @Override
+    public void handleCriticalEvent(DAGHistoryEvent event) throws IOException {
+      this.historyEvents.add(event.getHistoryEvent());
+    }
+
+    public List<HistoryEvent> getHistoryEvents() {
+      return historyEvents;
+    }
+
+    public void verifyHistoryEvent(int expectedTimes, HistoryEventType eventType) {
+      int actualCount = 0;
+      for (HistoryEvent event : historyEvents) {
+        if (event.getEventType() == eventType) {
+          actualCount ++;
+        }
+      }
+      assertEquals(expectedTimes, actualCount);
+    }
   }
 
-  /**
-   * restoreFromDAGInitializedEvent -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_FromInited() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
+  public static class MockInputInitializer extends InputInitializer {
 
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    public MockInputInitializer(InputInitializerContext initializerContext) {
+      super(initializerContext);
+    }
 
-    assertEquals(DAGState.RUNNING, dag.getState());
-    // send recover event to 2 root vertex
-    ArgumentCaptor<VertexEvent> eventCaptor =
-        ArgumentCaptor.forClass(VertexEvent.class);
-    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
-    List<VertexEvent> vertexEvents = eventCaptor.getAllValues();
-    assertEquals(2, vertexEvents.size());
-    for (VertexEvent vEvent : vertexEvents) {
-      assertTrue(vEvent instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vEvent;
-      assertEquals(VertexState.RUNNING, recoverEvent.getDesiredState());
+    @Override
+    public List<Event> initialize() throws Exception {
+      // sleep forever, block the initialization
+      while(true) {
+        Thread.sleep(1000);
+      }
+    }
+
+    @Override
+    public void handleInputInitializerEvent(List<InputInitializerEvent> events)
+        throws Exception {
     }
   }
 
-  /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGKillRequested -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_FromInitedToKilled() {
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGKillRequestEvent();
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.KILLED, dag.getState());
-    assertEquals(DAGTerminationCause.DAG_KILL, dag.getTerminationCause());
+  @BeforeClass
+  public static void beforeClass() {
+    MockDNSToSwitchMapping.initializeMockRackResolver();
   }
 
-  /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_FromStarted() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
+  @SuppressWarnings({ "unchecked", "rawtypes" })
+  @Before
+  public void setup() {
+    conf = new Configuration();
+    conf.setBoolean(TezConfiguration.TEZ_AM_CONTAINER_REUSE_ENABLED, false);
+    appAttemptId = ApplicationAttemptId.newInstance(
+        ApplicationId.newInstance(100, 1), 1);
+    dagId = TezDAGID.getInstance(appAttemptId.getApplicationId(), 1);
+    Assert.assertNotNull(dagId);
+    dagPlan = createDAGPlan();
+    dispatcher = new DrainDispatcher();
+    fsTokens = new Credentials();
+    appContext = mock(AppContext.class);
+    execService = mock(ListeningExecutorService.class);
+    thh = mock(TaskHeartbeatHandler.class);
+    final ListenableFuture<Void> mockFuture = mock(ListenableFuture.class);
+
+    Mockito.doAnswer(new Answer() {
+      public ListenableFuture<Void> answer(InvocationOnMock invocation) {
+        Object[] args = invocation.getArguments();
+        CallableEvent e = (CallableEvent) args[0];
+        dispatcher.getEventHandler().handle(e);
+        return mockFuture;
+      }
+    }).when(execService).submit((Callable<Void>) any());
+
+    doReturn(execService).when(appContext).getExecService();
+    historyEventHandler = new MockHistoryEventHandler(appContext);
+    aclManager = new ACLManager("amUser");
+    doReturn(conf).when(appContext).getAMConf();
+    doReturn(appAttemptId).when(appContext).getApplicationAttemptId();
+    doReturn(appAttemptId.getApplicationId()).when(appContext)
+        .getApplicationID();
+    doReturn(dagId).when(appContext).getCurrentDAGID();
+    doReturn(historyEventHandler).when(appContext).getHistoryHandler();
+    doReturn(aclManager).when(appContext).getAMACLManager();
+    doReturn(dagRecoveryData).when(appContext).getDAGRecoveryData();
+    dag = new DAGImpl(dagId, conf, dagPlan, dispatcher.getEventHandler(),
+        taskCommunicatorManagerInterface, fsTokens, clock, "user", thh,
+        appContext);
+    dag.entityUpdateTracker = new StateChangeNotifierForTest(dag);
+    doReturn(dag).when(appContext).getCurrentDAG();
+    ugi = mock(UserGroupInformation.class);
+    UserGroupInformation ugi =dag.getDagUGI();
+    doReturn(clusterInfo).when(appContext).getClusterInfo();
+    TaskSchedulerManager mockTaskScheduler = mock(TaskSchedulerManager.class);
+    doReturn(mockTaskScheduler).when(appContext).getTaskScheduler();
+    v1Id = TezVertexID.getInstance(dagId, 0);
+    t1v1Id = TezTaskID.getInstance(v1Id, 0);
+    ta1t1v1Id = TezTaskAttemptID.getInstance(t1v1Id, 0);
+    v2Id = TezVertexID.getInstance(dagId, 1);
+    t1v2Id = TezTaskID.getInstance(v2Id, 0);
+    ta1t1v2Id = TezTaskAttemptID.getInstance(t1v2Id, 0);
+
+    dispatcher.register(CallableEventType.class, new CallableEventDispatcher());
+    taskEventDispatcher = new TaskEventDispatcher();
+    dispatcher.register(TaskEventType.class, taskEventDispatcher);
+    taskAttemptEventDispatcher = new TaskAttemptEventDispatcher();
+    dispatcher.register(TaskAttemptEventType.class, taskAttemptEventDispatcher);
+    vertexEventDispatcher = new VertexEventDispatcher();
+    dispatcher.register(VertexEventType.class, vertexEventDispatcher);
+    dagEventDispatcher = new DagEventDispatcher();
+    dispatcher.register(DAGEventType.class, dagEventDispatcher);
+    dagFinishEventHandler = new DAGFinishEventHandler();
+    dispatcher.register(DAGAppMasterEventType.class, dagFinishEventHandler);
+    dispatcher.register(AMSchedulerEventType.class, new AMSchedulerEventDispatcher());
+    dispatcher.init(conf);
+    dispatcher.start();
+    doReturn(dispatcher.getEventHandler()).when(appContext).getEventHandler();
+    LogManager.getRootLogger().setLevel(Level.DEBUG);
+  }
 
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
 
-    assertEquals(DAGState.RUNNING, dag.getState());
-    // send recover event to 2 root vertex
-    ArgumentCaptor<VertexEvent> eventCaptor =
-        ArgumentCaptor.forClass(VertexEvent.class);
-    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
-    List<VertexEvent> vertexEvents = eventCaptor.getAllValues();
-    assertEquals(2, vertexEvents.size());
-    for (VertexEvent vEvent : vertexEvents) {
-      assertTrue(vEvent instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vEvent;
-      assertEquals(VertexState.RUNNING, recoverEvent.getDesiredState());
+  public static class RecoveryNotSupportedOutputCommitter extends OutputCommitter {
+
+    public RecoveryNotSupportedOutputCommitter(
+        OutputCommitterContext committerContext) {
+      super(committerContext);
+    }
+
+    @Override
+    public void initialize() throws Exception {
+    }
+
+    @Override
+    public void setupOutput() throws Exception {
+    }
+
+    @Override
+    public void commitOutput() throws Exception {
+    }
+
+    @Override
+    public void abortOutput(State finalState) throws Exception {
+    }
+    
+    @Override
+    public boolean isTaskRecoverySupported() {
+      return false;
     }
   }
 
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> restoreFromDAGKillRequested
-   * RecoverTransition
+   * v1     v2 
+   *   \    / 
+   *    \  / 
+   *     v3
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_FromStartedtoKilled() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromDAGKillRequestEvent();
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.KILLED, dag.getState());
-    assertEquals(DAGTerminationCause.DAG_KILL, dag.getTerminationCause());
-    // send recover event to all the vertices with desired state of KILLED
-    ArgumentCaptor<TezAbstractEvent> eventCaptor =
-        ArgumentCaptor.forClass(TezAbstractEvent.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<TezAbstractEvent> events = eventCaptor.getAllValues();
-    assertEquals(7, events.size());
-    for (int i=0;i<6;++i) {
-      TezAbstractEvent vEvent = events.get(i);
-      assertTrue(vEvent instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vEvent;
-      assertEquals(VertexState.KILLED, recoverEvent.getDesiredState());
-    }
-    assertTrue(events.get(6) instanceof DAGAppMasterEventDAGFinished);
+  private DAGPlan createDAGPlan() {
+    DAGPlan dag = DAGPlan
+        .newBuilder()
+        .setName("testverteximpl")
+        .addVertex(
+            VertexPlan
+                .newBuilder()
+                .setName("vertex1")
+                .setType(PlanVertexType.NORMAL)
+                .addInputs(RootInputLeafOutputProto.newBuilder().setName("input1")
+                    .setControllerDescriptor(TezEntityDescriptorProto.newBuilder()
+                        .setClassName(MockInputInitializer.class.getName()).build()))
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host1")
+                        .addRack("rack1").build())
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder().setNumTasks(-1)
+                        .setVirtualCores(4).setMemoryMb(1024).setJavaOpts("")
+                        .setTaskModule("x1.y1").build())
+                .addOutputs(
+                    DAGProtos.RootInputLeafOutputProto
+                        .newBuilder()
+                        .setIODescriptor(
+                            TezEntityDescriptorProto.newBuilder()
+                                .setClassName("output1").build())
+                        .setName("output1")
+                        .setControllerDescriptor(
+                            TezEntityDescriptorProto.newBuilder().setClassName(
+                                CountingOutputCommitter.class.getName())))
+                .addOutEdgeId("e1").build())
+        .addVertex(
+            VertexPlan
+                .newBuilder()
+                .setName("vertex2")
+                .setType(PlanVertexType.NORMAL)
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host2")
+                        .addRack("rack2").build())
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder().setNumTasks(1)
+                        .setVirtualCores(4).setMemoryMb(1024).setJavaOpts("")
+                        .setTaskModule("x2.y2").build())
+                .addOutputs(
+                    DAGProtos.RootInputLeafOutputProto
+                        .newBuilder()
+                        .setIODescriptor(
+                            TezEntityDescriptorProto.newBuilder()
+                                .setClassName("output2").build())
+                        .setName("output2")
+                        .setControllerDescriptor(
+                            TezEntityDescriptorProto.newBuilder().setClassName(
+                                RecoveryNotSupportedOutputCommitter.class.getName())))
+                .addOutEdgeId("e2").build())
+        .addVertex(
+            VertexPlan
+                .newBuilder()
+                .setName("vertex3")
+                .setType(PlanVertexType.NORMAL)
+                .setProcessorDescriptor(
+                    TezEntityDescriptorProto.newBuilder().setClassName("x3.y3"))
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host3")
+                        .addRack("rack3").build())
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder().setNumTasks(1)
+                        .setVirtualCores(4).setMemoryMb(1024)
+                        .setJavaOpts("foo").setTaskModule("x3.y3").build())
+                .addOutputs(
+                    DAGProtos.RootInputLeafOutputProto
+                        .newBuilder()
+                        .setIODescriptor(
+                            TezEntityDescriptorProto.newBuilder()
+                                .setClassName("output3").build())
+                        .setName("output3")
+                        .setControllerDescriptor(
+                            TezEntityDescriptorProto.newBuilder().setClassName(
+                                CountingOutputCommitter.class.getName())))
+                .addInEdgeId("e1").addInEdgeId("e2").build())
+        .addEdge(
+            EdgePlan
+                .newBuilder()
+                .setEdgeDestination(
+                    TezEntityDescriptorProto.newBuilder().setClassName("i2"))
+                .setInputVertexName("vertex1")
+                .setEdgeSource(
+                    TezEntityDescriptorProto.newBuilder().setClassName("o1"))
+                .setOutputVertexName("vertex3")
+                .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
+                .setId("e1")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL).build())
+        .addEdge(
+            EdgePlan
+                .newBuilder()
+                .setEdgeDestination(
+                    TezEntityDescriptorProto.newBuilder().setClassName("i3"))
+                .setInputVertexName("vertex2")
+                .setEdgeSource(
+                    TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+                .setOutputVertexName("vertex3")
+                .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
+                .setId("e2")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL).build())
+        .build();
+
+    return dag;
   }
 
+ 
+  @After
+  public void teardown() {
+    dispatcher.await();
+    dispatcher.stop();
+    execService.shutdownNow();
+    dagPlan = null;
+    if (dag != null) {
+      dag.entityUpdateTracker.stop();
+    }
+    dag = null;
+  }
+  
+  
+  ////////////////////////////////// DAG Recovery ///////////////////////////////////////////////////
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * restoreFromDAGFinishedEvent (SUCCEEDED) -> RecoverTransition
+   * RecoveryEvents: SummaryEvent_DAGFinishedEvent(SUCCEEDED)
+   * Recover dag to SUCCEEDED and all of its vertices to SUCCEEDED
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_Finished_SUCCEEDED() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.SUCCEEDED, dag.getState());
-    assertEquals(tezCounters, dag.getAllCounters());
-    // recover all the vertices to SUCCEED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
-    }
+  @Test(timeout=5000)
+  public void testDAGRecoverFromDesiredSucceeded() {
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, DAGState.SUCCEEDED, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+    assertEquals(DAGState.SUCCEEDED, dag.getState());
+    assertEquals(3, dag.getVertices().size());
+    assertEquals(VertexState.SUCCEEDED, dag.getVertex("vertex1").getState());
+    assertEquals(VertexState.SUCCEEDED, dag.getVertex("vertex2").getState());
+    assertEquals(VertexState.SUCCEEDED, dag.getVertex("vertex3").getState());
+    // DAG#initTime, startTime is not guaranteed to be recovered in this case
   }
-
+  
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * restoreFromDAGFinishedEvent(FAILED) -> RecoverTransition
+   * RecoveryEvents: SummaryEvent_DAGFinishedEvent(FAILED)
+   * Recover dag to FAILED and all of its vertices to FAILED
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_Finished_FAILED() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromDAGFinishedEvent(DAGState.FAILED);
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.FAILED, dag.getState());
-    assertEquals(tezCounters, dag.getAllCounters());
-    // recover all the vertices to FAILED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.FAILED, recoverEvent.getDesiredState());
-    }
+  @Test(timeout=5000)
+  public void testDAGRecoverFromDesiredFailed() {
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, DAGState.FAILED, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+    assertEquals(DAGState.FAILED, dag.getState());
+    assertEquals(3, dag.getVertices().size());
+    assertEquals(VertexState.FAILED, dag.getVertex("vertex1").getState());
+    assertEquals(VertexState.FAILED, dag.getVertex("vertex2").getState());
+    assertEquals(VertexState.FAILED, dag.getVertex("vertex3").getState());
+    // DAG#initTime, startTime is not guaranteed to be recovered in this case
   }
-
+  
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
-   * restoreFromDAGFinishedEvent -> RecoverTransition
+   * RecoveryEvents: SummaryEvent_DAGFinishedEvent(KILLED)
+   * Recover dag to KILLED and all of its vertices to KILLED
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_Finished_KILLED() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromDAGFinishedEvent(DAGState.KILLED);
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.KILLED, dag.getState());
-    assertEquals(tezCounters, dag.getAllCounters());
-    // recover all the vertices to KILLED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.KILLED, recoverEvent.getDesiredState());
-    }
+  @Test(timeout=5000)
+  public void testDAGRecoverFromDesiredKilled() {
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, DAGState.KILLED, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.KILLED, dagFinishedEvent.getDAGState());
+    assertEquals(DAGState.KILLED, dag.getState());
+    assertEquals(3, dag.getVertices().size());
+    assertEquals(VertexState.KILLED, dag.getVertex("vertex1").getState());
+    assertEquals(VertexState.KILLED, dag.getVertex("vertex2").getState());
+    assertEquals(VertexState.KILLED, dag.getVertex("vertex3").getState());
+    // DAG#initTime, startTime is not guaranteed to be recovered in this case
   }
-
+  
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent
-   * --> restoreFromDAGKillRequestEvent -->
-   * restoreFromDAGFinishedEvent -> RecoverTransition
+   * RecoveryEvents: SummaryEvent_DAGFinishedEvent(ERROR)
+   * Recover dag to ERROR and all of its vertices to ERROR
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_Finished_KILLED_WithKillRequest() {
-    // same behavior as without DAGKillRequestEvent because DAGFinishedEvent is seen
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromDAGKillRequestEvent();
-    restoreFromDAGFinishedEvent(DAGState.KILLED);
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.KILLED, dag.getState());
-    assertEquals(tezCounters, dag.getAllCounters());
-    // recover all the vertices to KILLED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.KILLED, recoverEvent.getDesiredState());
-    }
+  @Test(timeout=5000)
+  public void testDAGRecoverFromDesiredError() {
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, DAGState.ERROR, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.KILLED, dagFinishedEvent.getDAGState());
+    assertEquals(DAGState.ERROR, dag.getState());
+    assertEquals(3, dag.getVertices().size());
+    assertEquals(VertexState.ERROR, dag.getVertex("vertex1").getState());
+    assertEquals(VertexState.ERROR, dag.getVertex("vertex2").getState());
+    assertEquals(VertexState.ERROR, dag.getVertex("vertex3").getState());
+    // DAG#initTime, startTime is not guaranteed to be recovered in this case
   }
-
+  
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
-   * restoreFromDAGFinishedEvent -> RecoverTransition
+   * RecoveryEvents: DAGSubmittedEvent
+   * Recover it as normal dag execution
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_Finished_ERROR() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromDAGFinishedEvent(DAGState.ERROR);
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.ERROR, dag.getState());
-    assertEquals(tezCounters, dag.getAllCounters());
-    // recover all the vertices to KILLED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.FAILED, recoverEvent.getDesiredState());
-    }
+  @Test(timeout=5000)
+  public void testDAGRecoverFromNew() {
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.ERROR, dagFinishedEvent.getDAGState());
+    assertEquals(DAGState.RUNNING, dag.getState());
   }
-
+  
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * restoreFromDAG_COMMIT_STARTED -> RecoverTransition
+   * RecoveryEvents: DAGSubmittedEvent, DAGInitializedEvent
+   * Recover it as normal dag execution
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_COMMIT_STARTED() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromDAGCommitStartedEvent();
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.FAILED, dag.getState());
+  @Test(timeout=5000)
+  public void testDAGRecoverFromInited() {
+    DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagId, dagInitedTime, 
+        "user", "dagName", null);
+    doReturn(dagInitedEvent).when(dagRecoveryData).getDAGInitializedEvent();
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // recover all the vertices to SUCCEEDED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
-    }
+    assertEquals(DAGState.RUNNING, dag.getState());
+    assertEquals(dagInitedTime, dag.initTime);
+  }
+  
+  @Test(timeout=5000)
+  public void testDAGRecoverFromStarted() {
+    DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagId, dagInitedTime, 
+        "user", "dagName", null);
+    doReturn(dagInitedEvent).when(dagRecoveryData).getDAGInitializedEvent();
+    DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagId, dagStartedTime, "user", "dagName");
+    doReturn(dagStartedEvent).when(dagRecoveryData).getDAGStartedEvent();
+
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+    assertEquals(DAGState.RUNNING, dag.getState());
+    assertEquals(dagInitedTime, dag.initTime);
+    assertEquals(dagStartedTime, dag.startTime);
+  }
+ 
+  /////////////////////////////// Vertex Recovery /////////////////////////////////////////
+  
+  private void initMockDAGRecoveryDataForVertex() {    
+    DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagId, dagInitedTime, 
+        "user", "dagName", null);
+    DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagId, dagStartedTime, "user", "dagName");
+    doReturn(dagInitedEvent).when(dagRecoveryData).getDAGInitializedEvent();
+    doReturn(dagStartedEvent).when(dagRecoveryData).getDAGStartedEvent();
   }
 
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * restoreFromDAG_COMMIT_STARTED -> -> restoreFromDAGFinished (SUCCEEDED)->
-   * RecoverTransition
+   * RecoveryEvents:
+   *  DAG:  DAGInitedEvent -> DAGStartedEvent 
+   *  V1:   No any event
+   * 
+   * Reinitialize V1 again. 
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_COMMIT_STARTED_Finished_SUCCEEDED() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-
-    restoreFromDAGCommitStartedEvent();
-    restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.SUCCEEDED, dag.getState());
-
-    // recover all the vertices to SUCCEED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
-    }
-
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+  @Test(timeout=5000)
+  public void testVertexRecoverFromNew() {
+    initMockDAGRecoveryDataForVertex();
+    
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
+    assertEquals(DAGState.RUNNING, dag.getState());
+    // reinitialize v1 again
+    VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1");
+    VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2");
+    VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3");
+    assertEquals(VertexState.INITIALIZING, v1.getState());
+    assertEquals(VertexState.RUNNING, v2.getState());
+    assertEquals(VertexState.INITED, v3.getState());
   }
-
+  
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> RecoverTransition
+   * RecoveryEvents:
+   *  DAG:  DAGInitedEvent -> DAGStartedEvent 
+   *  V1:   VertexInitializedEvent
+   * 
+   * Reinitialize V1 again. 
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_GROUP_COMMIT_STARTED() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-    restoreFromVertexGroupCommitStartedEvent();
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.FAILED, dag.getState());
+  @Test(timeout=5000)
+  public void testVertexRecoverFromInited() {
+    initMockDAGRecoveryDataForVertex();
+    List<TezEvent> inputGeneratedTezEvents = new ArrayList<TezEvent>();
+    VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, 
+        "vertex1", 0L, v1InitedTime, 
+        v1NumTask, "", null, inputGeneratedTezEvents);
+    VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent,
+        null, null, null, null, false);
+    doReturn(vertexRecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id);
+    
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
 
-    // recover all the vertices to SUCCEEDED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
+    assertEquals(DAGState.RUNNING, dag.getState());
+    // reinitialize v1 again because its VertexManager is not completed
+    VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1");
+    VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2");
+    VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3");
+    assertEquals(VertexState.INITIALIZING, v1.getState());
+    assertEquals(VertexState.RUNNING, v2.getState());
+    assertEquals(VertexState.INITED, v3.getState());
+  }
+  
+  /**
+   * RecoveryEvents:
+   *  DAG:  DAGInitedEvent -> DAGStartedEvent 
+   *  V1:   VertexReconfigrationDoneEvent -> VertexInitializedEvent
+   * 
+   * V1 skip initialization. 
+   */
+  @Test//(timeout=5000)
+  public void testVertexRecoverFromInitedAndReconfigureDone() {
+    initMockDAGRecoveryDataForVertex();
+    List<TezEvent> inputGeneratedTezEvents = new ArrayList<TezEvent>();
+    VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, 
+        "vertex1", 0L, v1InitedTime, 
+        v1NumTask, "", null, inputGeneratedTezEvents);
+    VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, 
+        0L, v1NumTask, null, null, null, true);
+    VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent,
+        v1ReconfigureDoneEvent, null, null, new HashMap<TezTaskID, TaskRecoveryData>(), false);
+    doReturn(vertexRecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id);
+    
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
+    try {
+      Thread.sleep(1000);
+    } catch (InterruptedException e) {
+      e.printStackTrace();
     }
-
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+    VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1");
+    VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2");
+    VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3");
+    assertEquals(DAGState.RUNNING, dag.getState());
+    // v1 skip initialization
+    assertEquals(VertexState.RUNNING, v1.getState());
+    assertEquals(v1InitedTime, v1.initedTime);
+    assertEquals(v1NumTask, v1.getTotalTasks());
+    assertEquals(VertexState.RUNNING, v2.getState());
+    assertEquals(VertexState.RUNNING, v3.getState());
   }
-
+  
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
-   * RecoverTransition
+   * RecoveryEvents:
+   *  DAG:  DAGInitedEvent -> DAGStartedEvent 
+   *  V1:   VertexReconfigrationDoneEvent -> VertexInitializedEvent -> VertexStartedEvent
+   * 
+   * V1 skip initialization. 
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_GROUP_COMMIT_STARTED_FINISHED() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
-
-    restoreFromVertexGroupCommitStartedEvent();
-    restoreFromVertexGroupCommitFinishedEvent();
-
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+  @Test(timeout=5000)
+  public void testVertexRecoverFromStart() {
+    initMockDAGRecoveryDataForVertex();   
+    List<TezEvent> inputGeneratedTezEvents = new ArrayList<TezEvent>();
+    VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, 
+        "vertex1", 0L, v1InitedTime, 
+        v1NumTask, "", null, inputGeneratedTezEvents);
+    VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, 
+        0L, v1NumTask, null, null, null, true);
+    VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime);
+    VertexRecoveryData vertexRecoveryData = new VertexRecoveryData(v1InitedEvent,
+        v1ReconfigureDoneEvent, v1StartedEvent, null, new HashMap<TezTaskID, TaskRecoveryData>(), false);
+    doReturn(vertexRecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id);
+    
+    DAGEventRecoverEvent recoveryEvent = new DAGEventRecoverEvent(dagId, dagRecoveryData);
+    dag.handle(recoveryEvent);
+    dispatcher.await();
+
+    VertexImpl v1 = (VertexImpl)dag.getVertex("vertex1");
+    VertexImpl v2 = (VertexImpl)dag.getVertex("vertex2");
+    VertexImpl v3 = (VertexImpl)dag.getVertex("vertex3");
     assertEquals(DAGState.RUNNING, dag.getState());
+    // v1 skip initialization
+    assertEquals(VertexState.RUNNING, v1.getState());
+    assertEquals(v1InitedTime, v1.initedTime);
+    assertEquals(v1StartedTime, v1.startedTime);
+    assertEquals(v1NumTask, v1.getTotalTasks());
+    assertEquals(VertexState.RUNNING, v2.getState());
+    assertEquals(VertexState.RUNNING, v3.getState());
+  }
+  
+  /////////////////////////////// Task ////////////////////////////////////////////////////////////
+  
+  private void initMockDAGRecoveryDataForTask() {
+    List<TezEvent> inputGeneratedTezEvents = new ArrayList<TezEvent>();
+    VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, 
+        "vertex1", 0L, v1InitedTime, 
+        v1NumTask, "", null, inputGeneratedTezEvents);
+    Map<String, InputSpecUpdate> rootInputSpecs = new HashMap<String, InputSpecUpdate>();
+    VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, 
+        0L, v1NumTask, null, null, rootInputSpecs, true);
+    VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime);
+    VertexRecoveryData v1RecoveryData = new VertexRecoveryData(v1InitedEvent,
+        v1ReconfigureDoneEvent, v1StartedEvent, null, new HashMap<TezTaskID, TaskRecoveryData>(), false);
+    
+    DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagId, dagInitedTime, 
+        "user", "dagName", null);
+    DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagId, dagStartedTime, "user", "dagName");
+    doReturn(v1RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id);
+    doReturn(dagInitedEvent).when(dagRecoveryData).getDAGInitializedEvent();
+    doReturn(dagStartedEvent).when(dagRecoveryData).getDAGStartedEvent();
+  }
 
-    // send recover event to 2 root vertex
-    verify(mockEventHandler, times(2)).handle(
-        any(VertexEventRecoverVertex.class));
-    assertEquals(DAGState.RUNNING, dag.getState());
+  /**
+   * RecoveryEvent: TaskFinishedEvent(KILLED)
+   * Recover it to KILLED
+   */
+  @Test(timeout=5000)
+  public void testTaskRecoverFromKilled() {
+    initMockDAGRecoveryDataForTask();
+    TaskFinishedEvent taskFinishedEvent = new TaskFinishedEvent(t1v1Id, "v1",
+        0L, 0L, null, TaskState.KILLED, "", null, 4);
+    TaskRecoveryData taskRecoveryData = new TaskRecoveryData(null, taskFinishedEvent, null);
+    doReturn(taskRecoveryData).when(dagRecoveryData).getTaskRecoveryData(t1v1Id);
+    
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex(v1Id);
+    TaskImpl task = (TaskImpl)vertex1.getTask(t1v1Id);
+    assertEquals(TaskStateInternal.KILLED, task.getInternalState());
+    assertEquals(1, vertex1.getCompletedTasks());
+  }
+  
+  /**
+   * RecoveryEvent: TaskStartedEvent
+   * Recover it to Scheduled
+   */
+  @Test(timeout=5000)
+  public void testTaskRecoverFromStarted() {
+    initMockDAGRecoveryDataForTask();
+    TaskStartedEvent taskStartedEvent = new TaskStartedEvent(t1v1Id, "v1", 0L, 0L);
+    TaskRecoveryData taskRecoveryData = new TaskRecoveryData(taskStartedEvent, null, null);
+    doReturn(taskRecoveryData).when(dagRecoveryData).getTaskRecoveryData(t1v1Id);
+    
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+    
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex(v1Id);
+    TaskImpl task = (TaskImpl)vertex1.getTask(t1v1Id);
+    assertEquals(TaskStateInternal.SCHEDULED, task.getInternalState());
+  }
+  
+  /**
+   * RecoveryEvent: TaskStartedEvent -> TaskFinishedEvent
+   * Recover it to Scheduled
+   */
+  @Test(timeout=5000)
+  public void testTaskRecoverFromSucceeded() {
+    initMockDAGRecoveryDataForTask();
+    TaskStartedEvent taskStartedEvent = new TaskStartedEvent(t1v1Id, "v1", 0L, 0L);
+    TaskFinishedEvent taskFinishedEvent = new TaskFinishedEvent(t1v1Id, "v1",
+        0L, 0L, null, TaskState.SUCCEEDED, "", null, 4);
+    TaskAttemptStartedEvent taStartedEvent = new TaskAttemptStartedEvent(
+        ta1t1v1Id, "v1", 0L, mock(ContainerId.class), 
+        mock(NodeId.class), "", "", "");
+    List<TezEvent> taGeneratedEvents = new ArrayList<TezEvent>();
+    EventMetaData metadata = new EventMetaData(EventProducerConsumerType.OUTPUT,
+        "vertex1", "vertex3", ta1t1v2Id);
+    taGeneratedEvents.add(new TezEvent(DataMovementEvent.create(ByteBuffer.wrap(new byte[0])), metadata));
+    TaskAttemptFinishedEvent taFinishedEvent = new TaskAttemptFinishedEvent(
+        ta1t1v1Id, "v1", 0L, 0L, 
+        TaskAttemptState.SUCCEEDED, null, "", null, 
+        null, taGeneratedEvents, 0L, null, 0L);
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(taStartedEvent, taFinishedEvent);
+    Map<TezTaskAttemptID, TaskAttemptRecoveryData> taRecoveryDataMap =
+        new HashMap<TezTaskAttemptID, TaskAttemptRecoveryData>();
+    taRecoveryDataMap.put(ta1t1v1Id, taRecoveryData);
+    TaskRecoveryData taskRecoveryData = new TaskRecoveryData(taskStartedEvent, taskFinishedEvent, taRecoveryDataMap);
+    doReturn(taskRecoveryData).when(dagRecoveryData).getTaskRecoveryData(t1v1Id);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v1Id);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+    
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex(v1Id);
+    TaskImpl task = (TaskImpl)vertex1.getTask(t1v1Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v1Id);
+    assertEquals(VertexState.RUNNING, vertex1.getState());
+    assertEquals(1, vertex1.getCompletedTasks());
+    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+    assertEquals(TaskAttemptStateInternal.SUCCEEDED, taskAttempt.getInternalState());
+  }
+
+  /////////////////////////////// TaskAttempt Recovery /////////////////////////////////////////////////////
+  
+  private void initMockDAGRecoveryDataForTaskAttempt() {
+    TaskStartedEvent t1StartedEvent = new TaskStartedEvent(t1v1Id, "vertex1", 0L, t1StartedTime);
+    TaskRecoveryData taskRecoveryData = new TaskRecoveryData(t1StartedEvent, null, null);
+    Map<TezTaskID, TaskRecoveryData> taskRecoveryDataMap = new HashMap<TezTaskID, TaskRecoveryData>();
+    taskRecoveryDataMap.put(t1v1Id, taskRecoveryData);
+    
+    List<TezEvent> inputGeneratedTezEvents = new ArrayList<TezEvent>();
+    VertexInitializedEvent v1InitedEvent = new VertexInitializedEvent(v1Id, 
+        "vertex1", 0L, v1InitedTime, 
+        v1NumTask, "", null, inputGeneratedTezEvents);
+    Map<String, InputSpecUpdate> rootInputSpecs = new HashMap<String, InputSpecUpdate>();
+    VertexConfigurationDoneEvent v1ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v1Id, 
+        0L, v1NumTask, null, null, rootInputSpecs, true);
+    VertexStartedEvent v1StartedEvent = new VertexStartedEvent(v1Id, 0L, v1StartedTime);
+    VertexRecoveryData v1RecoveryData = new VertexRecoveryData(v1InitedEvent,
+        v1ReconfigureDoneEvent, v1StartedEvent, null, taskRecoveryDataMap, false);
+    
+    DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagId, dagInitedTime, 
+        "user", "dagName", null);
+    DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagId, dagStartedTime, "user", "dagName");
+    doReturn(v1RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v1Id);
+    doReturn(dagInitedEvent).when(dagRecoveryData).getDAGInitializedEvent();
+    doReturn(dagStartedEvent).when(dagRecoveryData).getDAGStartedEvent();
   }
 
   /**
-   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
-   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
-   * restoreFromDAG_Finished -> RecoverTransition
+   * RecoveryEvents: TaskAttemptFinishedEvent (FAILED)
+   * Recover it to FAILED
    */
-  @Test(timeout = 5000)
-  public void testDAGRecovery_GROUP_COMMIT_Finished() {
-    assertNewState();
-    restoreFromDAGInitializedEvent();
-    restoreFromDAGStartedEvent();
+  @Test(timeout=5000)
+  public void testTARecoverFromNewToFailed() {
+    initMockDAGRecoveryDataForTaskAttempt();
+    TaskAttemptFinishedEvent taFinishedEvent = new TaskAttemptFinishedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, ta1FinishedTime, 
+        TaskAttemptState.FAILED, TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED, "", null, 
+        null, null, 0L, null, 0L);
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(null, taFinishedEvent);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v1Id);
+    
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+
+    TaskImpl task = (TaskImpl)dag.getVertex(v1Id).getTask(t1v1Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v1Id);
+    assertEquals(TaskAttemptStateInternal.FAILED, taskAttempt.getInternalState());
+    assertEquals(TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED, taskAttempt.getTerminationCause());
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_STARTED);
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_FINISHED);
+    assertEquals(1, task.failedAttempts);
+    // new task attempt is scheduled
+    assertEquals(2, task.getAttempts().size());
+    assertEquals(ta1FinishedTime, taskAttempt.getFinishTime());
+  }
+  
+  /**
+   * RecoveryEvents: TaskAttemptFinishedEvent (KILLED)
+   * Recover it to KILLED
+   */
+  @Test(timeout=5000)
+  public void testTARecoverFromNewToKilled() {
+    initMockDAGRecoveryDataForTaskAttempt();
+    TaskAttemptFinishedEvent taFinishedEvent = new TaskAttemptFinishedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, ta1FinishedTime, 
+        TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT, "", null, 
+        null, null, 0L, null, 0L);
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(null, taFinishedEvent);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v1Id);
+    
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+
+    TaskImpl task = (TaskImpl)dag.getVertex(v1Id).getTask(t1v1Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v1Id);
+    assertEquals(TaskAttemptStateInternal.KILLED, taskAttempt.getInternalState());
+    assertEquals(TaskAttemptTerminationCause.TERMINATED_BY_CLIENT, taskAttempt.getTerminationCause());
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_STARTED);
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_FINISHED);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(ta1FinishedTime, taskAttempt.getFinishTime());
+  }
+  
+  /**
+   * RecoveryEvents: TaskAttemptStartedEvent
+   * Recover it to KILLED
+   */
+  @Test(timeout=5000)
+  public void testTARecoverFromRunning() {
+    initMockDAGRecoveryDataForTaskAttempt();
+    TaskAttemptStartedEvent taStartedEvent = new TaskAttemptStartedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, mock(ContainerId.class), 
+        mock(NodeId.class), "", "", "");
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(taStartedEvent, null);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v1Id);
+    
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+
+    TaskImpl task = (TaskImpl)dag.getVertex(v1Id).getTask(t1v1Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v1Id);
+    assertEquals(TaskAttemptStateInternal.KILLED, taskAttempt.getInternalState());
+    assertEquals(TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, taskAttempt.getTerminationCause());
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_STARTED);
+    historyEventHandler.verifyHistoryEvent(1, HistoryEventType.TASK_ATTEMPT_FINISHED);
+    assertEquals(ta1LaunchTime, taskAttempt.getLaunchTime());
+  }
 
-    restoreFromVertexGroupCommitStartedEvent();
-    restoreFromVertexGroupCommitFinishedEvent();
-    restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+  /**
+   * RecoveryEvents: TaskAttemptStartedEvent -> TaskAttemptFinishedEvent (SUCCEEDED)
+   * Recover it to SUCCEEDED
+   */
+  @Test(timeout=5000)
+  public void testTARecoverFromSucceeded() {
+    initMockDAGRecoveryDataForTaskAttempt();
+    TaskAttemptStartedEvent taStartedEvent = new TaskAttemptStartedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, mock(ContainerId.class), 
+        mock(NodeId.class), "", "", "");
+    List<TezEvent> taGeneratedEvents = new ArrayList<TezEvent>();
+    taGeneratedEvents.add(new TezEvent(DataMovementEvent.create(ByteBuffer.wrap(new byte[0])), null));
+    TaskAttemptFinishedEvent taFinishedEvent = new TaskAttemptFinishedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, ta1FinishedTime, 
+        TaskAttemptState.SUCCEEDED, null, "", null, 
+        null, taGeneratedEvents, 0L, null, 0L);
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(taStartedEvent, taFinishedEvent);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v1Id);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+    
+    TaskImpl task = (TaskImpl)dag.getVertex(v1Id).getTask(t1v1Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v1Id);
+    assertEquals(TaskAttemptStateInternal.SUCCEEDED, taskAttempt.getInternalState());
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_FINISHED);
+    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+    assertEquals(ta1LaunchTime, taskAttempt.getLaunchTime());
+    assertEquals(ta1FinishedTime, taskAttempt.getFinishTime());
+  }
 
-    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
-    assertEquals(DAGState.SUCCEEDED, dag.getState());
-    assertEquals(tezCounters, dag.getAllCounters());
-    // recover all the vertices to SUCCEEDED
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(7)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    int i = 0;
-    for (; i < 6; ++i) {
-      assertTrue(events.get(i) instanceof VertexEventRecoverVertex);
-      VertexEventRecoverVertex recoverEvent =
-          (VertexEventRecoverVertex) events.get(i);
-      assertEquals(VertexState.SUCCEEDED, recoverEvent.getDesiredState());
-    }
+  /**
+   * RecoveryEvents: TaskAttemptStartedEvent -> TaskAttemptFinishedEvent (SUCCEEDED)
+   * Recovered it SUCCEEDED, but task schedule new task attempt
+   * V2's committer is not recovery supported
+   */
+  @Test//(timeout=5000)
+  public void testTARecoverFromSucceeded_OutputCommitterRecoveryNotSupported() {
+    initMockDAGRecoveryDataForTaskAttempt();
+    // set up v2 recovery data
+    // ta1t1v2: TaskAttemptStartedEvent -> TaskAttemptFinishedEvent(SUCCEEDED)
+    // t1v2: TaskStartedEvent
+    // v2: VertexInitializedEvent -> VertexConfigurationDoneEvent -> VertexStartedEvent
+    TaskAttemptStartedEvent taStartedEvent = new TaskAttemptStartedEvent(
+        ta1t1v2Id, "vertex2", ta1LaunchTime, mock(ContainerId.class), 
+        mock(NodeId.class), "", "", "");
+    List<TezEvent> taGeneratedEvents = new ArrayList<TezEvent>();
+    EventMetaData metadata = new EventMetaData(EventProducerConsumerType.OUTPUT,
+        "vertex2", "vertex3", ta1t1v2Id);
+    taGeneratedEvents.add(new TezEvent(DataMovementEvent.create(ByteBuffer.wrap(new byte[0])), metadata));
+    TaskAttemptFinishedEvent taFinishedEvent = new TaskAttemptFinishedEvent(
+        ta1t1v2Id, "vertex2", ta1LaunchTime, ta1FinishedTime, 
+        TaskAttemptState.SUCCEEDED, null, "", null, 
+        null, taGeneratedEvents, 0L, null, 0L);
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(taStartedEvent, taFinishedEvent);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v2Id);   
+    Map<TezTaskAttemptID, TaskAttemptRecoveryData> taRecoveryDataMap =
+        new HashMap<TezTaskAttemptID, TaskAttemptRecoveryData>();
+    taRecoveryDataMap.put(ta1t1v2Id, taRecoveryData);
+ 
+    TaskStartedEvent t1StartedEvent = new TaskStartedEvent(t1v2Id, "vertex2", 0L, t1StartedTime);
+    TaskRecoveryData taskRecoveryData = new TaskRecoveryData(t1StartedEvent, null, taRecoveryDataMap);
+    Map<TezTaskID, TaskRecoveryData> taskRecoveryDataMap = new HashMap<TezTaskID, TaskRecoveryData>();
+    taskRecoveryDataMap.put(t1v2Id, taskRecoveryData);
+    doReturn(taskRecoveryData).when(dagRecoveryData).getTaskRecoveryData(t1v2Id);
+
+    VertexInitializedEvent v2InitedEvent = new VertexInitializedEvent(v2Id, 
+        "vertex2", 0L, v1InitedTime, 
+        v1NumTask, "", null, null);
+    VertexConfigurationDoneEvent v2ReconfigureDoneEvent = new VertexConfigurationDoneEvent(v2Id, 
+        0L, v1NumTask, null, null, null, false);
+    VertexStartedEvent v2StartedEvent = new VertexStartedEvent(v2Id, 0L, v1StartedTime);
+    VertexRecoveryData v2RecoveryData = new VertexRecoveryData(v2InitedEvent,
+        v2ReconfigureDoneEvent, v2StartedEvent, null, taskRecoveryDataMap, false);
+    doReturn(v2RecoveryData).when(dagRecoveryData).getVertexRecoveryData(v2Id);
+
+
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+    
+    TaskImpl task = (TaskImpl)dag.getVertex(v2Id).getTask(t1v2Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v2Id);
+    assertEquals(TaskAttemptStateInternal.SUCCEEDED, taskAttempt.getInternalState());
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_FINISHED);
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    // new task attempt is scheduled
+    assertEquals(2, task.getAttempts().size());
+    assertEquals(ta1LaunchTime, taskAttempt.getLaunchTime());
+    assertEquals(ta1FinishedTime, taskAttempt.getFinishTime());
+  }
 
-    // send DAGAppMasterEventDAGFinished at last
-    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
-    DAGAppMasterEventDAGFinished dagFinishedEvent =
-        (DAGAppMasterEventDAGFinished) events.get(i);
-    assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+  /**
+   * RecoveryEvents: TaskAttemptStartedEvent -> TaskAttemptFinishedEvent (FAILED)
+   * Recover it to FAILED
+   */
+  @Test(timeout=5000)
+  public void testTARecoverFromFailed() {
+    initMockDAGRecoveryDataForTaskAttempt();
+    TaskAttemptStartedEvent taStartedEvent = new TaskAttemptStartedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, mock(ContainerId.class), 
+        mock(NodeId.class), "", "", "");
+    TaskAttemptFinishedEvent taFinishedEvent = new TaskAttemptFinishedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, ta1FinishedTime, 
+        TaskAttemptState.FAILED, TaskAttemptTerminationCause.INPUT_READ_ERROR, "", null, 
+        null, null, 0L, null, 0L);
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(taStartedEvent, taFinishedEvent);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v1Id);
+    
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+    
+    TaskImpl task = (TaskImpl)dag.getVertex(v1Id).getTask(t1v1Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v1Id);
+    assertEquals(TaskAttemptStateInternal.FAILED, taskAttempt.getInternalState());
+    assertEquals(TaskAttemptTerminationCause.INPUT_READ_ERROR, taskAttempt.getTerminationCause());
+    assertEquals(TaskStateInternal.SCHEDULED, task.getInternalState());
+    assertEquals(2, task.getAttempts().size());
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_FINISHED);
+    assertEquals(ta1LaunchTime, taskAttempt.getLaunchTime());
+    assertEquals(ta1FinishedTime, taskAttempt.getFinishTime());
   }
 
+  /**
+   * RecoveryEvents: TaskAttemptStartedEvent -> TaskAttemptFinishedEvent (KILLED)
+   * Recover it to KILLED
+   */
+  @Test(timeout=5000)
+  public void testTARecoverFromKilled() {
+    initMockDAGRecoveryDataForTaskAttempt();
+    TaskAttemptStartedEvent taStartedEvent = new TaskAttemptStartedEvent(
+        ta1t1v1Id, "v1", ta1LaunchTime, mock(ContainerId.class), 
+        mock(NodeId.class), "", "", "");
+    TaskAttemptFinishedEvent taFinishedEvent = new TaskAttemptFinishedEvent(
+        ta1t1v1Id, "v1", ta1FinishedTime, ta1FinishedTime, 
+        TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT, "", null, 
+        null, null, 0L, null, 0L);
+    TaskAttemptRecoveryData taRecoveryData = new TaskAttemptRecoveryData(taStartedEvent, taFinishedEvent);
+    doReturn(taRecoveryData).when(dagRecoveryData).getTaskAttemptRecoveryData(ta1t1v1Id);
+    
+    dag.handle(new DAGEventRecoverEvent(dagId, dagRecoveryData));
+    dispatcher.await();
+    
+    TaskImpl task = (TaskImpl)dag.getVertex(v1Id).getTask(t1v1Id);
+    TaskAttemptImpl taskAttempt = (TaskAttemptImpl)task.getAttempt(ta1t1v1Id);
+    assertEquals(TaskAttemptStateInternal.KILLED, taskAttempt.getInternalState());
+    assertEquals(TaskAttemptTerminationCause.TERMINATED_BY_CLIENT, taskAttempt.getTerminationCause());
+    historyEventHandler.verifyHistoryEvent(0, HistoryEventType.TASK_ATTEMPT_FINISHED);
+    assertEquals(ta1LaunchTime, taskAttempt.getLaunchTime());
+    assertEquals(ta1FinishedTime, taskAttempt.getFinishTime());
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
index 17295cd..6dd578f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttempt.java
@@ -57,6 +57,8 @@ import org.apache.hadoop.yarn.event.Event;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
 import org.apache.tez.common.MockDNSToSwitchMapping;
 import org.apache.tez.dag.api.TaskCommunicator;
 import org.apache.tez.dag.api.TaskLocationHint;
@@ -137,6 +139,7 @@ public class TestTaskAttempt {
     mockTask = mock(Task.class);
     HistoryEventHandler mockHistHandler = mock(HistoryEventHandler.class);
     doReturn(mockHistHandler).when(appCtx).getHistoryHandler();
+    LogManager.getRootLogger().setLevel(Level.DEBUG);
   }
 
   @Test(timeout = 5000)
@@ -1556,7 +1559,7 @@ public class TestTaskAttempt {
         Resource resource, ContainerContext containerContext, boolean leafVertex) {
       super(taskId, attemptNumber, eventHandler, tal, conf,
           clock, taskHeartbeatHandler, appContext,
-          isRescheduled, resource, containerContext, leafVertex, mockTask);
+          isRescheduled, resource, containerContext, leafVertex, mockTask, null);
       when(mockTask.getTaskLocationHint()).thenReturn(locationHint);
     }
 


[6/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
index e17a4d4..95ff0cd 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventHandler.java
@@ -64,7 +64,10 @@ public class HistoryEventHandler extends CompositeService {
     addService(historyLoggingService);
 
     if (recoveryEnabled) {
-      recoveryService = new RecoveryService(context);
+      String recoveryServiceClass = conf.get(TezConfiguration.TEZ_AM_RECOVERY_SERVICE_CLASS,
+          TezConfiguration.TEZ_AM_RECOVERY_SERVICE_CLASS_DEFAULT);
+      recoveryService = ReflectionUtils.createClazzInstance(recoveryServiceClass,
+          new Class[]{AppContext.class}, new Object[] {context});
       addService(recoveryService);
     }
     super.serviceInit(conf);

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
index d791d9e..4e56e9f 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/HistoryEventType.java
@@ -29,7 +29,7 @@ public enum HistoryEventType {
   DAG_KILL_REQUEST,
   VERTEX_INITIALIZED,
   VERTEX_STARTED,
-  VERTEX_PARALLELISM_UPDATED,
+  VERTEX_CONFIGURE_DONE,
   VERTEX_FINISHED,
   TASK_STARTED,
   TASK_FINISHED,
@@ -37,7 +37,6 @@ public enum HistoryEventType {
   TASK_ATTEMPT_FINISHED,
   CONTAINER_LAUNCHED,
   CONTAINER_STOPPED,
-  VERTEX_DATA_MOVEMENT_EVENTS_GENERATED,
   DAG_COMMIT_STARTED,
   VERTEX_COMMIT_STARTED,
   VERTEX_GROUP_COMMIT_STARTED,

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/RecoveryConverters.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/RecoveryConverters.java b/tez-dag/src/main/java/org/apache/tez/dag/history/RecoveryConverters.java
new file mode 100644
index 0000000..bab713d
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/RecoveryConverters.java
@@ -0,0 +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.tez.dag.history;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+
+public class RecoveryConverters {
+
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java
index 7d83db2..21b8719 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskAttemptFinishedEvent.java
@@ -34,10 +34,13 @@ import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl.DataEventDependencyInfo;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.utils.TezEventUtils;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.DataEventDependencyInfoProto;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.TaskAttemptFinishedProto;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 public class TaskAttemptFinishedEvent implements HistoryEvent {
 
@@ -55,6 +58,7 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
   private TezCounters tezCounters;
   private TaskAttemptTerminationCause error;
   private List<DataEventDependencyInfo> dataEvents;
+  private List<TezEvent> taGeneratedEvents;
   
   public TaskAttemptFinishedEvent(TezTaskAttemptID taId,
       String vertexName,
@@ -63,7 +67,8 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
       TaskAttemptState state,
       TaskAttemptTerminationCause error,
       String diagnostics, TezCounters counters, 
-      List<DataEventDependencyInfo> dataEvents, 
+      List<DataEventDependencyInfo> dataEvents,
+      List<TezEvent> taGeneratedEvents,
       long creationTime, 
       TezTaskAttemptID creationCausalTA, 
       long allocationTime) {
@@ -79,6 +84,7 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
     this.tezCounters = counters;
     this.error = error;
     this.dataEvents = dataEvents;
+    this.taGeneratedEvents = taGeneratedEvents;
   }
 
   public TaskAttemptFinishedEvent() {
@@ -103,7 +109,7 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
     return dataEvents;
   }
   
-  public TaskAttemptFinishedProto toProto() {
+  public TaskAttemptFinishedProto toProto() throws IOException {
     TaskAttemptFinishedProto.Builder builder =
         TaskAttemptFinishedProto.newBuilder();
     builder.setTaskAttemptId(taskAttemptId.toString())
@@ -129,10 +135,15 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
         builder.addDataEvents(DataEventDependencyInfo.toProto(info));
       }
     }
+    if (taGeneratedEvents != null && !taGeneratedEvents.isEmpty()) {
+      for (TezEvent event : taGeneratedEvents) {
+        builder.addTaGeneratedEvents(TezEventUtils.toProto(event));
+      }
+    }
     return builder.build();
   }
 
-  public void fromProto(TaskAttemptFinishedProto proto) {
+  public void fromProto(TaskAttemptFinishedProto proto) throws IOException {
     this.taskAttemptId = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
     this.state = TaskAttemptState.values()[proto.getState()];
     this.creationTime = proto.getCreationTime();
@@ -158,6 +169,12 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
         this.dataEvents.add(DataEventDependencyInfo.fromProto(protoEvent));
       }
     }
+    if (proto.getTaGeneratedEventsCount() > 0) {
+      this.taGeneratedEvents = Lists.newArrayListWithCapacity(proto.getTaGeneratedEventsCount());
+      for (TezEventProto eventProto : proto.getTaGeneratedEventsList()) {
+        this.taGeneratedEvents.add(TezEventUtils.fromProto(eventProto));
+      }
+    }
   }
 
   @Override
@@ -236,4 +253,7 @@ public class TaskAttemptFinishedEvent implements HistoryEvent {
     return creationCausalTA;
   }
 
+  public List<TezEvent> getTAGeneratedEvents() {
+    return taGeneratedEvents;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java
new file mode 100644
index 0000000..4ad1c63
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexConfigurationDoneEvent.java
@@ -0,0 +1,211 @@
+/**
+ * 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.history.events;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.apache.tez.dag.api.DagTypeConverters;
+import org.apache.tez.dag.api.EdgeProperty;
+import org.apache.tez.dag.api.VertexLocationHint;
+import org.apache.tez.dag.history.HistoryEvent;
+import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.EdgeManagerDescriptorProto;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.RootInputSpecUpdateProto;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexConfigurationDoneProto;
+import org.apache.tez.runtime.api.InputSpecUpdate;
+
+import com.google.common.collect.Maps;
+
+public class VertexConfigurationDoneEvent implements HistoryEvent {
+
+  private TezVertexID vertexID;
+  private long reconfigureDoneTime;
+  private int numTasks;
+  private VertexLocationHint vertexLocationHint;
+  private Map<String, EdgeProperty> sourceEdgeProperties;
+  private Map<String, InputSpecUpdate> rootInputSpecUpdates;
+  private boolean setParallelismCalledFlag;
+
+  public VertexConfigurationDoneEvent() {  
+  }
+
+  public VertexConfigurationDoneEvent(TezVertexID vertexID,
+      long reconfigureDoneTime, int numTasks,
+      VertexLocationHint vertexLocationHint,
+      Map<String, EdgeProperty> sourceEdgeProperties,
+      Map<String, InputSpecUpdate> rootInputSpecUpdates,
+      boolean setParallelismCalledFlag) {
+    super();
+    this.vertexID = vertexID;
+    this.numTasks = numTasks;
+    this.vertexLocationHint = vertexLocationHint;
+    this.sourceEdgeProperties = sourceEdgeProperties;
+    this.rootInputSpecUpdates = rootInputSpecUpdates;
+    this.setParallelismCalledFlag = setParallelismCalledFlag;
+  }
+
+  @Override
+  public HistoryEventType getEventType() {
+    return HistoryEventType.VERTEX_CONFIGURE_DONE;
+  }
+
+  @Override
+  public boolean isRecoveryEvent() {
+    return true;
+  }
+
+  @Override
+  public boolean isHistoryEvent() {
+    return true;
+  }
+
+  public VertexConfigurationDoneProto toProto() {
+    VertexConfigurationDoneProto.Builder builder =
+        VertexConfigurationDoneProto.newBuilder();
+    builder.setVertexId(vertexID.toString())
+        .setReconfigureDoneTime(reconfigureDoneTime)
+        .setSetParallelismCalledFlag(setParallelismCalledFlag)
+        .setNumTasks(numTasks);
+
+    if (vertexLocationHint != null) {
+      builder.setVertexLocationHint(DagTypeConverters.convertVertexLocationHintToProto(
+            this.vertexLocationHint));
+    }
+    if (sourceEdgeProperties != null) {
+      for (Entry<String, EdgeProperty> entry :
+        sourceEdgeProperties.entrySet()) {
+        EdgeManagerDescriptorProto.Builder edgeMgrBuilder =
+            EdgeManagerDescriptorProto.newBuilder();
+        edgeMgrBuilder.setEdgeName(entry.getKey());
+        edgeMgrBuilder.setEdgeProperty(DagTypeConverters.convertToProto(entry.getValue()));
+        builder.addEdgeManagerDescriptors(edgeMgrBuilder.build());
+      }
+    }
+    if (rootInputSpecUpdates != null) {
+      for (Entry<String, InputSpecUpdate> entry : rootInputSpecUpdates.entrySet()) {
+        RootInputSpecUpdateProto.Builder rootInputSpecUpdateBuilder = RootInputSpecUpdateProto
+            .newBuilder();
+        rootInputSpecUpdateBuilder.setInputName(entry.getKey());
+        rootInputSpecUpdateBuilder.setForAllWorkUnits(entry.getValue().isForAllWorkUnits());
+        rootInputSpecUpdateBuilder.addAllNumPhysicalInputs(entry.getValue()
+            .getAllNumPhysicalInputs());
+        builder.addRootInputSpecUpdates(rootInputSpecUpdateBuilder.build());
+      }
+    }
+    return builder.build();
+  }
+
+  public void fromProto(VertexConfigurationDoneProto proto) {
+    this.vertexID = TezVertexID.fromString(proto.getVertexId());
+    this.reconfigureDoneTime = proto.getReconfigureDoneTime();
+    this.setParallelismCalledFlag = proto.getSetParallelismCalledFlag();
+    this.numTasks = proto.getNumTasks();
+    if (proto.hasVertexLocationHint()) {
+      this.vertexLocationHint = DagTypeConverters.convertVertexLocationHintFromProto(
+          proto.getVertexLocationHint());
+    }
+    if (proto.getEdgeManagerDescriptorsCount() > 0) {
+      this.sourceEdgeProperties = new HashMap<String, EdgeProperty>(
+          proto.getEdgeManagerDescriptorsCount());
+      for (EdgeManagerDescriptorProto edgeManagerProto :
+        proto.getEdgeManagerDescriptorsList()) {
+        EdgeProperty edgeProperty =
+            DagTypeConverters.convertFromProto(
+                edgeManagerProto.getEdgeProperty());
+        sourceEdgeProperties.put(edgeManagerProto.getEdgeName(),
+            edgeProperty);
+      }
+    }
+    if (proto.getRootInputSpecUpdatesCount() > 0) {
+      this.rootInputSpecUpdates = Maps.newHashMap();
+      for (RootInputSpecUpdateProto rootInputSpecUpdateProto : proto.getRootInputSpecUpdatesList()) {
+        InputSpecUpdate specUpdate;
+        if (rootInputSpecUpdateProto.getForAllWorkUnits()) {
+          specUpdate = InputSpecUpdate
+              .createAllTaskInputSpecUpdate(rootInputSpecUpdateProto.getNumPhysicalInputs(0));
+        } else {
+          specUpdate = InputSpecUpdate
+              .createPerTaskInputSpecUpdate(rootInputSpecUpdateProto.getNumPhysicalInputsList());
+        }
+        this.rootInputSpecUpdates.put(rootInputSpecUpdateProto.getInputName(), specUpdate);
+      }
+    }
+  }
+
+  @Override
+  public void toProtoStream(OutputStream outputStream) throws IOException {
+    toProto().writeDelimitedTo(outputStream);
+  }
+
+  @Override
+  public void fromProtoStream(InputStream inputStream) throws IOException {
+    VertexConfigurationDoneProto proto = VertexConfigurationDoneProto.parseDelimitedFrom(inputStream);
+    if (proto == null) {
+      throw new IOException("No data found in stream");
+    }
+    fromProto(proto);
+  }
+
+  @Override
+  public String toString() {
+    return "vertexId=" + vertexID
+        + ", reconfigureDoneTime=" + reconfigureDoneTime
+        + ", numTasks=" + numTasks
+        + ", vertexLocationHint=" +
+        (vertexLocationHint == null? "null" : vertexLocationHint)
+        + ", edgeManagersCount=" +
+        (sourceEdgeProperties == null? "null" : sourceEdgeProperties.size())
+        + ", rootInputSpecUpdateCount="
+        + (rootInputSpecUpdates == null ? "null" : rootInputSpecUpdates.size())
+        + ", setParallelismCalledFlag=" + setParallelismCalledFlag;
+  }
+
+  public TezVertexID getVertexID() {
+    return this.vertexID;
+  }
+
+  public int getNumTasks() {
+    return numTasks;
+  }
+
+  public VertexLocationHint getVertexLocationHint() {
+    return vertexLocationHint;
+  }
+
+  public Map<String, EdgeProperty> getSourceEdgeProperties() {
+    return sourceEdgeProperties;
+  }
+
+  public Map<String, InputSpecUpdate> getRootInputSpecUpdates() {
+    return rootInputSpecUpdates;
+  }
+
+  public long getReconfigureDoneTime() {
+    return reconfigureDoneTime;
+  }
+
+  public boolean isSetParallelismCalled() {
+    return setParallelismCalledFlag;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java
index 8947b5f..ec8f3e1 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitFinishedEvent.java
@@ -21,28 +21,35 @@ package org.apache.tez.dag.history.events;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Collection;
 
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.SummaryEvent;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.recovery.records.RecoveryProtos;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexGroupCommitFinishedProto;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+
 public class VertexGroupCommitFinishedEvent implements HistoryEvent, SummaryEvent {
 
   private TezDAGID dagID;
   private String vertexGroupName;
+  private Collection<TezVertexID> vertexIds;
   private long commitFinishTime;
 
   public VertexGroupCommitFinishedEvent() {
   }
 
   public VertexGroupCommitFinishedEvent(TezDAGID dagID,
-      String vertexGroupName, long commitFinishTime) {
+      String vertexGroupName, Collection<TezVertexID> vertexIds, long commitFinishTime) {
     this.dagID = dagID;
     this.vertexGroupName = vertexGroupName;
+    this.vertexIds = vertexIds;
     this.commitFinishTime = commitFinishTime;
   }
 
@@ -62,15 +69,28 @@ public class VertexGroupCommitFinishedEvent implements HistoryEvent, SummaryEven
   }
 
   public VertexGroupCommitFinishedProto toProto() {
+    Collection<String> vertexIdsStr = Collections2.transform(vertexIds, new Function<TezVertexID, String>(){
+      @Override
+      public String apply(TezVertexID vertexId) {
+        return vertexId.toString();
+      }
+    });
     return VertexGroupCommitFinishedProto.newBuilder()
         .setDagId(dagID.toString())
         .setVertexGroupName(vertexGroupName)
+        .addAllVertexIds(vertexIdsStr)
         .build();
   }
 
   public void fromProto(VertexGroupCommitFinishedProto proto) {
     this.dagID = TezDAGID.fromString(proto.getDagId());
     this.vertexGroupName = proto.getVertexGroupName();
+    this.vertexIds = Collections2.transform(proto.getVertexIdsList(), new Function<String, TezVertexID>() {
+      @Override
+      public TezVertexID apply(String input) {
+        return TezVertexID.fromString(input);
+      }
+    });
   }
 
   @Override
@@ -124,4 +144,8 @@ public class VertexGroupCommitFinishedEvent implements HistoryEvent, SummaryEven
     return dagID;
   }
 
+  public Collection<TezVertexID> getVertexIds() {
+    return vertexIds;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java
index c388957..3de355c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexGroupCommitStartedEvent.java
@@ -21,28 +21,35 @@ package org.apache.tez.dag.history.events;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.util.Collection;
 
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
 import org.apache.tez.dag.history.SummaryEvent;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.recovery.records.RecoveryProtos;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexGroupCommitStartedProto;
 
+import com.google.common.base.Function;
+import com.google.common.collect.Collections2;
+
 public class VertexGroupCommitStartedEvent implements HistoryEvent, SummaryEvent {
 
   private TezDAGID dagID;
   private String vertexGroupName;
+  private Collection<TezVertexID> vertexIds;
   private long commitStartTime;
 
   public VertexGroupCommitStartedEvent() {
   }
 
   public VertexGroupCommitStartedEvent(TezDAGID dagID,
-      String vertexGroupName, long commitStartTime) {
+      String vertexGroupName, Collection<TezVertexID> vertexIds, long commitStartTime) {
     this.dagID = dagID;
     this.vertexGroupName = vertexGroupName;
+    this.vertexIds = vertexIds;
     this.commitStartTime = commitStartTime;
   }
 
@@ -62,15 +69,28 @@ public class VertexGroupCommitStartedEvent implements HistoryEvent, SummaryEvent
   }
 
   public VertexGroupCommitStartedProto toProto() {
+    Collection<String> vertexIdsStr = Collections2.transform(vertexIds, new Function<TezVertexID, String>(){
+      @Override
+      public String apply(TezVertexID vertexId) {
+        return vertexId.toString();
+      }
+    });
     return VertexGroupCommitStartedProto.newBuilder()
         .setDagId(dagID.toString())
         .setVertexGroupName(vertexGroupName)
+        .addAllVertexIds(vertexIdsStr)
         .build();
   }
 
   public void fromProto(VertexGroupCommitStartedProto proto) {
     this.dagID = TezDAGID.fromString(proto.getDagId());
     this.vertexGroupName = proto.getVertexGroupName();
+    this.vertexIds = Collections2.transform(proto.getVertexIdsList(), new Function<String, TezVertexID>() {
+      @Override
+      public TezVertexID apply(String input) {
+        return TezVertexID.fromString(input);
+      }
+    });
   }
 
   @Override
@@ -124,4 +144,7 @@ public class VertexGroupCommitStartedEvent implements HistoryEvent, SummaryEvent
     return dagID;
   }
 
+  public Collection<TezVertexID> getVertexIds() {
+    return vertexIds;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
index 01e0d3c..052908b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexInitializedEvent.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.util.LinkedHashMap;
+import java.util.List;
 import java.util.Map;
 
 import org.apache.tez.dag.api.DagTypeConverters;
@@ -31,9 +32,14 @@ import org.apache.tez.dag.api.RootInputLeafOutput;
 import org.apache.tez.dag.api.records.DAGProtos.RootInputLeafOutputProto;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
+import org.apache.tez.dag.history.utils.TezEventUtils;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.recovery.records.RecoveryProtos;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexInitializedProto;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+import com.google.common.collect.Lists;
 
 public class VertexInitializedEvent implements HistoryEvent {
 
@@ -44,6 +50,7 @@ public class VertexInitializedEvent implements HistoryEvent {
   private int numTasks;
   private String processorName;
   private Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> additionalInputs;
+  private List<TezEvent> initGeneratedEvents;
 
   public VertexInitializedEvent() {
   }
@@ -51,7 +58,8 @@ public class VertexInitializedEvent implements HistoryEvent {
   public VertexInitializedEvent(TezVertexID vertexId,
       String vertexName, long initRequestedTime, long initedTime,
       int numTasks, String processorName,
-      Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> additionalInputs) {
+      Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> additionalInputs,
+      List<TezEvent> initGeneratedEvents) {
     this.vertexName = vertexName;
     this.vertexID = vertexId;
     this.initRequestedTime = initRequestedTime;
@@ -59,6 +67,7 @@ public class VertexInitializedEvent implements HistoryEvent {
     this.numTasks = numTasks;
     this.processorName = processorName;
     this.additionalInputs = additionalInputs;
+    this.initGeneratedEvents = initGeneratedEvents;
   }
 
   @Override
@@ -76,7 +85,7 @@ public class VertexInitializedEvent implements HistoryEvent {
     return true;
   }
 
-  public RecoveryProtos.VertexInitializedProto toProto() {
+  public RecoveryProtos.VertexInitializedProto toProto() throws IOException {
     VertexInitializedProto.Builder builder = VertexInitializedProto.newBuilder();
     if (additionalInputs != null
       && !additionalInputs.isEmpty()) {
@@ -94,6 +103,11 @@ public class VertexInitializedEvent implements HistoryEvent {
         builder.addInputs(inputBuilder.build());
       }
     }
+    if (initGeneratedEvents != null && !initGeneratedEvents.isEmpty()) {
+      for (TezEvent event : initGeneratedEvents) {
+        builder.addInitGeneratedEvents(TezEventUtils.toProto(event));
+      }
+    }
     return builder.setVertexId(vertexID.toString())
         .setVertexName(vertexName)
         .setInitRequestedTime(initRequestedTime)
@@ -102,7 +116,7 @@ public class VertexInitializedEvent implements HistoryEvent {
         .build();
   }
 
-  public void fromProto(RecoveryProtos.VertexInitializedProto proto) {
+  public void fromProto(RecoveryProtos.VertexInitializedProto proto) throws IOException {
     this.vertexID = TezVertexID.fromString(proto.getVertexId());
     this.vertexName = proto.getVertexName();
     this.initRequestedTime = proto.getInitRequestedTime();
@@ -123,6 +137,14 @@ public class VertexInitializedEvent implements HistoryEvent {
         additionalInputs.put(input.getName(), input);
       }
     }
+    int eventCount = proto.getInitGeneratedEventsCount();
+    if (eventCount > 0) {
+      this.initGeneratedEvents = Lists.newArrayListWithCapacity(eventCount);
+    }
+    for (TezEventProto eventProto :
+        proto.getInitGeneratedEventsList()) {
+      this.initGeneratedEvents.add(TezEventUtils.fromProto(eventProto));
+    }
   }
 
   @Override
@@ -149,7 +171,9 @@ public class VertexInitializedEvent implements HistoryEvent {
         + ", numTasks=" + numTasks
         + ", processorName=" + processorName
         + ", additionalInputsCount="
-        + (additionalInputs != null ? additionalInputs.size() : 0);
+        + (additionalInputs != null ? additionalInputs.size() : 0)
+        + ", initGeneratedEventsCount="
+        + (initGeneratedEvents != null ? initGeneratedEvents.size() : 0);
   }
 
   public TezVertexID getVertexID() {
@@ -181,4 +205,7 @@ public class VertexInitializedEvent implements HistoryEvent {
     return vertexName;
   }
 
+  public List<TezEvent> getInitGeneratedEvents() {
+    return initGeneratedEvents;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexParallelismUpdatedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexParallelismUpdatedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexParallelismUpdatedEvent.java
deleted file mode 100644
index 456e2a5..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexParallelismUpdatedEvent.java
+++ /dev/null
@@ -1,204 +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.history.events;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.apache.tez.dag.api.DagTypeConverters;
-import org.apache.tez.dag.api.EdgeProperty;
-import org.apache.tez.dag.api.VertexLocationHint;
-import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.HistoryEventType;
-import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.EdgeManagerDescriptorProto;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.RootInputSpecUpdateProto;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexParallelismUpdatedProto;
-import org.apache.tez.runtime.api.InputSpecUpdate;
-
-import com.google.common.collect.Maps;
-
-public class VertexParallelismUpdatedEvent implements HistoryEvent {
-
-  private TezVertexID vertexID;
-  private int numTasks;
-  private int oldNumTasks;
-  private VertexLocationHint vertexLocationHint;
-  private Map<String, EdgeProperty> sourceEdgeProperties;
-  private Map<String, InputSpecUpdate> rootInputSpecUpdates;
-  private long updateTime;
-
-  public VertexParallelismUpdatedEvent() {
-  }
-
-  public VertexParallelismUpdatedEvent(TezVertexID vertexID,
-      int numTasks, VertexLocationHint vertexLocationHint,
-      Map<String, EdgeProperty> sourceEdgeProperties,
-      Map<String, InputSpecUpdate> rootInputSpecUpdates, int oldNumTasks) {
-    this.vertexID = vertexID;
-    this.numTasks = numTasks;
-    this.vertexLocationHint = vertexLocationHint;
-    this.sourceEdgeProperties = sourceEdgeProperties;
-    this.rootInputSpecUpdates = rootInputSpecUpdates;
-    this.updateTime = System.currentTimeMillis();
-    this.oldNumTasks = oldNumTasks;
-  }
-
-  @Override
-  public HistoryEventType getEventType() {
-    return HistoryEventType.VERTEX_PARALLELISM_UPDATED;
-  }
-
-  @Override
-  public boolean isRecoveryEvent() {
-    return true;
-  }
-
-  @Override
-  public boolean isHistoryEvent() {
-    return true;
-  }
-
-  public VertexParallelismUpdatedProto toProto() {
-    VertexParallelismUpdatedProto.Builder builder =
-        VertexParallelismUpdatedProto.newBuilder();
-    builder.setVertexId(vertexID.toString())
-        .setNumTasks(numTasks);
-    if (vertexLocationHint != null) {
-      builder.setVertexLocationHint(DagTypeConverters.convertVertexLocationHintToProto(
-            this.vertexLocationHint));
-    }
-    if (sourceEdgeProperties != null) {
-      for (Entry<String, EdgeProperty> entry :
-        sourceEdgeProperties.entrySet()) {
-        EdgeManagerDescriptorProto.Builder edgeMgrBuilder =
-            EdgeManagerDescriptorProto.newBuilder();
-        edgeMgrBuilder.setEdgeName(entry.getKey());
-        edgeMgrBuilder.setEdgeProperty(DagTypeConverters.convertToProto(entry.getValue()));
-        builder.addEdgeManagerDescriptors(edgeMgrBuilder.build());
-      }
-    }
-    if (rootInputSpecUpdates != null) {
-      for (Entry<String, InputSpecUpdate> entry : rootInputSpecUpdates.entrySet()) {
-        RootInputSpecUpdateProto.Builder rootInputSpecUpdateBuilder = RootInputSpecUpdateProto
-            .newBuilder();
-        rootInputSpecUpdateBuilder.setInputName(entry.getKey());
-        rootInputSpecUpdateBuilder.setForAllWorkUnits(entry.getValue().isForAllWorkUnits());
-        rootInputSpecUpdateBuilder.addAllNumPhysicalInputs(entry.getValue()
-            .getAllNumPhysicalInputs());
-        builder.addRootInputSpecUpdates(rootInputSpecUpdateBuilder.build());
-      }
-    }
-    return builder.build();
-  }
-
-  public void fromProto(VertexParallelismUpdatedProto proto) {
-    this.vertexID = TezVertexID.fromString(proto.getVertexId());
-    this.numTasks = proto.getNumTasks();
-    if (proto.hasVertexLocationHint()) {
-      this.vertexLocationHint = DagTypeConverters.convertVertexLocationHintFromProto(
-          proto.getVertexLocationHint());
-    }
-    if (proto.getEdgeManagerDescriptorsCount() > 0) {
-      this.sourceEdgeProperties = new HashMap<String, EdgeProperty>(
-          proto.getEdgeManagerDescriptorsCount());
-      for (EdgeManagerDescriptorProto edgeManagerProto :
-        proto.getEdgeManagerDescriptorsList()) {
-        EdgeProperty edgeProperty =
-            DagTypeConverters.convertFromProto(
-                edgeManagerProto.getEdgeProperty());
-        sourceEdgeProperties.put(edgeManagerProto.getEdgeName(),
-            edgeProperty);
-      }
-    }
-    if (proto.getRootInputSpecUpdatesCount() > 0) {
-      this.rootInputSpecUpdates = Maps.newHashMap();
-      for (RootInputSpecUpdateProto rootInputSpecUpdateProto : proto.getRootInputSpecUpdatesList()) {
-        InputSpecUpdate specUpdate;
-        if (rootInputSpecUpdateProto.getForAllWorkUnits()) {
-          specUpdate = InputSpecUpdate
-              .createAllTaskInputSpecUpdate(rootInputSpecUpdateProto.getNumPhysicalInputs(0));
-        } else {
-          specUpdate = InputSpecUpdate
-              .createPerTaskInputSpecUpdate(rootInputSpecUpdateProto.getNumPhysicalInputsList());
-        }
-        this.rootInputSpecUpdates.put(rootInputSpecUpdateProto.getInputName(), specUpdate);
-      }
-    }
-  }
-
-  @Override
-  public void toProtoStream(OutputStream outputStream) throws IOException {
-    toProto().writeDelimitedTo(outputStream);
-  }
-
-  @Override
-  public void fromProtoStream(InputStream inputStream) throws IOException {
-    VertexParallelismUpdatedProto proto = VertexParallelismUpdatedProto.parseDelimitedFrom(inputStream);
-    if (proto == null) {
-      throw new IOException("No data found in stream");
-    }
-    fromProto(proto);
-  }
-
-  @Override
-  public String toString() {
-    return "vertexId=" + vertexID
-        + ", numTasks=" + numTasks
-        + ", vertexLocationHint=" +
-        (vertexLocationHint == null? "null" : vertexLocationHint)
-        + ", edgeManagersCount=" +
-        (sourceEdgeProperties == null? "null" : sourceEdgeProperties.size()
-        + ", rootInputSpecUpdateCount="
-        + (rootInputSpecUpdates == null ? "null" : rootInputSpecUpdates.size()));
-  }
-
-  public TezVertexID getVertexID() {
-    return this.vertexID;
-  }
-
-  public int getNumTasks() {
-    return numTasks;
-  }
-
-  public VertexLocationHint getVertexLocationHint() {
-    return vertexLocationHint;
-  }
-
-  public Map<String, EdgeProperty> getSourceEdgeProperties() {
-    return sourceEdgeProperties;
-  }
-  
-  public Map<String, InputSpecUpdate> getRootInputSpecUpdates() {
-    return rootInputSpecUpdates;
-  }
-
-  public long getUpdateTime() {
-    return updateTime;
-  }
-
-  public int getOldNumTasks() {
-    return oldNumTasks;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java
deleted file mode 100644
index 6f44f33..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java
+++ /dev/null
@@ -1,224 +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.history.events;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.util.EnumSet;
-import java.util.List;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.tez.common.ProtoConverters;
-import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.HistoryEventType;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.dag.recovery.records.RecoveryProtos;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.TezDataMovementEventProto;
-import org.apache.tez.dag.recovery.records.RecoveryProtos.VertexDataMovementEventsGeneratedProto;
-import org.apache.tez.runtime.api.Event;
-import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
-import org.apache.tez.runtime.api.events.DataMovementEvent;
-import org.apache.tez.runtime.api.events.InputDataInformationEvent;
-import org.apache.tez.runtime.api.events.InputInitializerEvent;
-import org.apache.tez.runtime.api.impl.EventMetaData;
-import org.apache.tez.runtime.api.impl.EventType;
-import org.apache.tez.runtime.api.impl.TezEvent;
-
-import com.google.common.collect.Lists;
-
-// TODO PreCommit - rename this to VertexRecoverableEventGeneratedEvent
-public class VertexRecoverableEventsGeneratedEvent implements HistoryEvent {
-
-  private static final Logger LOG = LoggerFactory.getLogger(
-      VertexRecoverableEventsGeneratedEvent.class);
-  private List<TezEvent> events;
-  private TezVertexID vertexID;
-
-  public VertexRecoverableEventsGeneratedEvent(TezVertexID vertexID,
-                                               List<TezEvent> events) {
-    this.vertexID = vertexID;
-    this.events = Lists.newArrayListWithCapacity(events.size());
-    for (TezEvent event : events) {
-      if (EnumSet.of(EventType.DATA_MOVEMENT_EVENT,
-          EventType.COMPOSITE_DATA_MOVEMENT_EVENT,
-          EventType.ROOT_INPUT_DATA_INFORMATION_EVENT,
-          EventType.ROOT_INPUT_INITIALIZER_EVENT)
-              .contains(event.getEventType())) {
-        this.events.add(event);
-      }
-    }
-    if (events.isEmpty()) {
-      throw new RuntimeException("Invalid creation of VertexDataMovementEventsGeneratedEvent"
-        + ", no data movement/information events provided");
-    }
-  }
-
-  public VertexRecoverableEventsGeneratedEvent() {
-  }
-
-  @Override
-  public HistoryEventType getEventType() {
-    return HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED;
-  }
-
-  @Override
-  public boolean isRecoveryEvent() {
-    return true;
-  }
-
-  @Override
-  public boolean isHistoryEvent() {
-    return false;
-  }
-
-  static RecoveryProtos.EventMetaDataProto convertEventMetaDataToProto(
-      EventMetaData eventMetaData) {
-    RecoveryProtos.EventMetaDataProto.Builder builder =
-        RecoveryProtos.EventMetaDataProto.newBuilder()
-        .setProducerConsumerType(eventMetaData.getEventGenerator().ordinal())
-        .setEdgeVertexName(eventMetaData.getEdgeVertexName())
-        .setTaskVertexName(eventMetaData.getTaskVertexName());
-    if (eventMetaData.getTaskAttemptID() != null) {
-        builder.setTaskAttemptId(eventMetaData.getTaskAttemptID().toString());
-    }
-    return builder.build();
-  }
-
-  static EventMetaData convertEventMetaDataFromProto(
-      RecoveryProtos.EventMetaDataProto proto) {
-    TezTaskAttemptID attemptID = null;
-    if (proto.hasTaskAttemptId()) {
-      attemptID = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
-    }
-    return new EventMetaData(
-        EventMetaData.EventProducerConsumerType.values()[proto.getProducerConsumerType()],
-        proto.getTaskVertexName(),
-        proto.getEdgeVertexName(),
-        attemptID);
-  }
-
-  public VertexDataMovementEventsGeneratedProto toProto() {
-    List<TezDataMovementEventProto> tezEventProtos = null;
-    if (events != null) {
-      tezEventProtos = Lists.newArrayListWithCapacity(events.size());
-      for (TezEvent event : events) {
-        TezDataMovementEventProto.Builder evtBuilder =
-            TezDataMovementEventProto.newBuilder();
-        if (event.getEventType().equals(EventType.COMPOSITE_DATA_MOVEMENT_EVENT)) {
-          evtBuilder.setCompositeDataMovementEvent(
-              ProtoConverters.convertCompositeDataMovementEventToProto(
-                  (CompositeDataMovementEvent) event.getEvent()));
-        } else if (event.getEventType().equals(EventType.DATA_MOVEMENT_EVENT)) {
-          evtBuilder.setDataMovementEvent(
-              ProtoConverters.convertDataMovementEventToProto(
-                  (DataMovementEvent) event.getEvent()));
-        } else if (event.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
-          evtBuilder.setRootInputDataInformationEvent(
-              ProtoConverters.convertRootInputDataInformationEventToProto(
-                  (InputDataInformationEvent) event.getEvent()));
-        } else if (event.getEventType().equals(EventType.ROOT_INPUT_INITIALIZER_EVENT)) {
-          evtBuilder.setInputInitializerEvent(ProtoConverters
-              .convertRootInputInitializerEventToProto((InputInitializerEvent) event.getEvent()));
-        }
-        if (event.getSourceInfo() != null) {
-          evtBuilder.setSourceInfo(convertEventMetaDataToProto(event.getSourceInfo()));
-        }
-        if (event.getDestinationInfo() != null) {
-          evtBuilder.setDestinationInfo(convertEventMetaDataToProto(event.getDestinationInfo()));
-        }
-        evtBuilder.setEventTime(event.getEventReceivedTime());
-        tezEventProtos.add(evtBuilder.build());
-      }
-    }
-    return VertexDataMovementEventsGeneratedProto.newBuilder()
-        .setVertexId(vertexID.toString())
-        .addAllTezDataMovementEvent(tezEventProtos)
-        .build();
-  }
-
-  public void fromProto(VertexDataMovementEventsGeneratedProto proto) {
-    this.vertexID = TezVertexID.fromString(proto.getVertexId());
-    int eventCount = proto.getTezDataMovementEventCount();
-    if (eventCount > 0) {
-      this.events = Lists.newArrayListWithCapacity(eventCount);
-    }
-    for (TezDataMovementEventProto eventProto :
-        proto.getTezDataMovementEventList()) {
-      Event evt = null;
-      if (eventProto.hasCompositeDataMovementEvent()) {
-        evt = ProtoConverters.convertCompositeDataMovementEventFromProto(
-            eventProto.getCompositeDataMovementEvent());
-      } else if (eventProto.hasDataMovementEvent()) {
-        evt = ProtoConverters.convertDataMovementEventFromProto(
-            eventProto.getDataMovementEvent());
-      } else if (eventProto.hasRootInputDataInformationEvent()) {
-        evt = ProtoConverters.convertRootInputDataInformationEventFromProto(
-            eventProto.getRootInputDataInformationEvent());
-      } else if (eventProto.hasInputInitializerEvent()) {
-        evt = ProtoConverters.convertRootInputInitializerEventFromProto(
-            eventProto.getInputInitializerEvent());
-      }
-      EventMetaData sourceInfo = null;
-      EventMetaData destinationInfo = null;
-      if (eventProto.hasSourceInfo()) {
-        sourceInfo = convertEventMetaDataFromProto(eventProto.getSourceInfo());
-      }
-      if (eventProto.hasDestinationInfo()) {
-        destinationInfo = convertEventMetaDataFromProto(eventProto.getDestinationInfo());
-      }
-      TezEvent tezEvent = new TezEvent(evt, sourceInfo, eventProto.getEventTime());
-      tezEvent.setDestinationInfo(destinationInfo);
-      this.events.add(tezEvent);
-    }
-  }
-
-  @Override
-  public void toProtoStream(OutputStream outputStream) throws IOException {
-    toProto().writeDelimitedTo(outputStream);
-  }
-
-  @Override
-  public void fromProtoStream(InputStream inputStream) throws IOException {
-    VertexDataMovementEventsGeneratedProto proto =
-        VertexDataMovementEventsGeneratedProto.parseDelimitedFrom(inputStream);
-    if (proto == null) {
-      throw new IOException("No data found in stream");
-    }
-    fromProto(proto);
-  }
-
-  @Override
-  public String toString() {
-    return "vertexId=" + vertexID.toString()
-        + ", eventCount=" + (events != null ? events.size() : "null");
-
-  }
-
-  public TezVertexID getVertexID() {
-    return this.vertexID;
-  }
-
-  public List<TezEvent> getTezEvents() {
-    return this.events;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
index bf63045..c4e7e5b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/logging/impl/HistoryEventJsonConversion.java
@@ -42,7 +42,7 @@ import org.apache.tez.dag.history.events.TaskFinishedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.history.events.VertexFinishedEvent;
 import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
 import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.history.logging.EntityTypes;
 import org.apache.tez.dag.history.utils.DAGUtils;
@@ -108,13 +108,12 @@ public class HistoryEventJsonConversion {
       case TASK_ATTEMPT_FINISHED:
         jsonObject = convertTaskAttemptFinishedEvent((TaskAttemptFinishedEvent) historyEvent);
         break;
-      case VERTEX_PARALLELISM_UPDATED:
-        jsonObject = convertVertexParallelismUpdatedEvent((VertexParallelismUpdatedEvent) historyEvent);
+      case VERTEX_CONFIGURE_DONE:
+        jsonObject = convertVertexReconfigureDoneEvent((VertexConfigurationDoneEvent) historyEvent);
         break;
       case DAG_RECOVERED:
         jsonObject = convertDAGRecoveredEvent((DAGRecoveredEvent) historyEvent);
         break;
-      case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
       case VERTEX_COMMIT_STARTED:
       case VERTEX_GROUP_COMMIT_STARTED:
       case VERTEX_GROUP_COMMIT_FINISHED:
@@ -662,7 +661,6 @@ public class HistoryEventJsonConversion {
     JSONObject otherInfo = new JSONObject();
     otherInfo.put(ATSConstants.START_TIME, event.getStartTime());
     otherInfo.put(ATSConstants.SCHEDULED_TIME, event.getScheduledTime());
-
     jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
 
     return jsonObject;
@@ -705,6 +703,41 @@ public class HistoryEventJsonConversion {
     return jsonObject;
   }
 
+  private static JSONObject convertVertexReconfigureDoneEvent(VertexConfigurationDoneEvent event) throws JSONException {
+    JSONObject jsonObject = new JSONObject();
+    jsonObject.put(ATSConstants.ENTITY, event.getVertexID().toString());
+    jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name());
+
+    // Events
+    JSONArray events = new JSONArray();
+    JSONObject updateEvent = new JSONObject();
+    updateEvent.put(ATSConstants.TIMESTAMP, event.getReconfigureDoneTime());
+    updateEvent.put(ATSConstants.EVENT_TYPE,
+        HistoryEventType.VERTEX_CONFIGURE_DONE.name());
+
+    JSONObject eventInfo = new JSONObject();
+    eventInfo.put(ATSConstants.NUM_TASKS, event.getNumTasks());
+    if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) {
+      JSONObject updatedEdgeManagers = new JSONObject();
+      for (Entry<String, EdgeProperty> entry :
+          event.getSourceEdgeProperties().entrySet()) {
+        updatedEdgeManagers.put(entry.getKey(),
+            new JSONObject(DAGUtils.convertEdgeProperty(entry.getValue())));
+      }
+      eventInfo.put(ATSConstants.UPDATED_EDGE_MANAGERS, updatedEdgeManagers);
+    }
+    updateEvent.put(ATSConstants.EVENT_INFO, eventInfo);
+    events.put(updateEvent);
+    jsonObject.put(ATSConstants.EVENTS, events);
+
+    // Other info
+    JSONObject otherInfo = new JSONObject();
+    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
+
+    // TODO add more on all other updated information
+    return jsonObject;
+  }
+
   private static JSONObject convertVertexInitializedEvent(VertexInitializedEvent event) throws JSONException {
     JSONObject jsonObject = new JSONObject();
     jsonObject.put(ATSConstants.ENTITY, event.getVertexID().toString());
@@ -773,42 +806,4 @@ public class HistoryEventJsonConversion {
     return jsonObject;
   }
 
-  private static JSONObject convertVertexParallelismUpdatedEvent(
-      VertexParallelismUpdatedEvent event) throws JSONException {
-    JSONObject jsonObject = new JSONObject();
-    jsonObject.put(ATSConstants.ENTITY, event.getVertexID().toString());
-    jsonObject.put(ATSConstants.ENTITY_TYPE, EntityTypes.TEZ_VERTEX_ID.name());
-
-    // Events
-    JSONArray events = new JSONArray();
-    JSONObject updateEvent = new JSONObject();
-    updateEvent.put(ATSConstants.TIMESTAMP, event.getUpdateTime());
-    updateEvent.put(ATSConstants.EVENT_TYPE,
-        HistoryEventType.VERTEX_PARALLELISM_UPDATED.name());
-
-    JSONObject eventInfo = new JSONObject();
-    eventInfo.put(ATSConstants.OLD_NUM_TASKS, event.getOldNumTasks());
-    eventInfo.put(ATSConstants.NUM_TASKS, event.getNumTasks());
-    if (event.getSourceEdgeProperties() != null && !event.getSourceEdgeProperties().isEmpty()) {
-      JSONObject updatedEdgeManagers = new JSONObject();
-      for (Entry<String, EdgeProperty> entry :
-          event.getSourceEdgeProperties().entrySet()) {
-        updatedEdgeManagers.put(entry.getKey(),
-            new JSONObject(DAGUtils.convertEdgeProperty(entry.getValue())));
-      }
-      eventInfo.put(ATSConstants.UPDATED_EDGE_MANAGERS, updatedEdgeManagers);
-    }
-    updateEvent.put(ATSConstants.EVENT_INFO, eventInfo);
-    events.put(updateEvent);
-    jsonObject.put(ATSConstants.EVENTS, events);
-
-    // Other info
-    JSONObject otherInfo = new JSONObject();
-    otherInfo.put(ATSConstants.NUM_TASKS, event.getNumTasks());
-    jsonObject.put(ATSConstants.OTHER_INFO, otherInfo);
-
-    // TODO add more on all other updated information
-    return jsonObject;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
index 585050d..fed4f3d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/recovery/RecoveryService.java
@@ -50,7 +50,7 @@ import com.google.common.annotations.VisibleForTesting;
 public class RecoveryService extends AbstractService {
 
   private static final Logger LOG = LoggerFactory.getLogger(RecoveryService.class);
-  private final AppContext appContext;
+  protected final AppContext appContext;
 
   public static final String RECOVERY_FATAL_OCCURRED_DIR =
       "RecoveryFatalErrorOccurred";
@@ -73,7 +73,7 @@ public class RecoveryService extends AbstractService {
   private Set<TezDAGID> completedDAGs = new HashSet<TezDAGID>();
   private Set<TezDAGID> skippedDAGs = new HashSet<TezDAGID>();
 
-  private Thread eventHandlingThread;
+  public Thread eventHandlingThread;
   private AtomicBoolean stopped = new AtomicBoolean(false);
   private AtomicBoolean started = new AtomicBoolean(false);
   private int eventCounter = 0;
@@ -374,7 +374,7 @@ public class RecoveryService extends AbstractService {
     }
   }
 
-  private void handleSummaryEvent(TezDAGID dagID,
+  protected void handleSummaryEvent(TezDAGID dagID,
       HistoryEventType eventType,
       SummaryEvent summaryEvent) throws IOException {
     if (LOG.isDebugEnabled()) {
@@ -506,4 +506,8 @@ public class RecoveryService extends AbstractService {
       Thread.yield();
     }
   }
+
+  public void setStopped(boolean stopped) {
+    this.stopped.set(stopped);
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java
new file mode 100644
index 0000000..cc89b9f
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/utils/TezEventUtils.java
@@ -0,0 +1,131 @@
+/**
+ * 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.history.utils;
+
+import java.io.IOException;
+import org.apache.tez.common.ProtoConverters;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.recovery.records.RecoveryProtos;
+import org.apache.tez.dag.recovery.records.RecoveryProtos.TezEventProto;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.api.events.VertexManagerEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventType;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+public class TezEventUtils {
+
+  public static TezEventProto toProto(TezEvent event) throws IOException {
+    TezEventProto.Builder evtBuilder =
+        TezEventProto.newBuilder();
+    if (event.getEventType().equals(EventType.COMPOSITE_DATA_MOVEMENT_EVENT)) {
+      evtBuilder.setCompositeDataMovementEvent(
+          ProtoConverters.convertCompositeDataMovementEventToProto(
+              (CompositeDataMovementEvent) event.getEvent()));
+    } else if (event.getEventType().equals(EventType.DATA_MOVEMENT_EVENT)) {
+      evtBuilder.setDataMovementEvent(
+          ProtoConverters.convertDataMovementEventToProto(
+              (DataMovementEvent) event.getEvent()));
+    } else if (event.getEventType().equals(EventType.ROOT_INPUT_INITIALIZER_EVENT)) {
+      evtBuilder.setInputInitializerEvent(ProtoConverters
+          .convertRootInputInitializerEventToProto((InputInitializerEvent) event.getEvent()));
+    } else if (event.getEventType().equals(EventType.VERTEX_MANAGER_EVENT)) {
+      evtBuilder.setVmEvent(ProtoConverters
+          .convertVertexManagerEventToProto((VertexManagerEvent)event.getEvent()));
+    } else if (event.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
+      evtBuilder.setRootInputDataInformationEvent(
+          ProtoConverters.convertRootInputDataInformationEventToProto(
+              (InputDataInformationEvent) event.getEvent()));
+    } else {
+      throw new IOException("Unsupported TezEvent type:" + event.getEventType());
+    }
+
+    if (event.getSourceInfo() != null) {
+      evtBuilder.setSourceInfo(convertEventMetaDataToProto(event.getSourceInfo()));
+    }
+    if (event.getDestinationInfo() != null) {
+      evtBuilder.setDestinationInfo(convertEventMetaDataToProto(event.getDestinationInfo()));
+    }
+    evtBuilder.setEventTime(event.getEventReceivedTime());
+    return evtBuilder.build();
+  }
+
+  public static TezEvent fromProto(TezEventProto eventProto) throws IOException {
+    Event evt = null;
+    if (eventProto.hasCompositeDataMovementEvent()) {
+      evt = ProtoConverters.convertCompositeDataMovementEventFromProto(
+          eventProto.getCompositeDataMovementEvent());
+    } else if (eventProto.hasDataMovementEvent()) {
+      evt = ProtoConverters.convertDataMovementEventFromProto(
+          eventProto.getDataMovementEvent());
+    } else if (eventProto.hasInputInitializerEvent()) {
+      evt = ProtoConverters.convertRootInputInitializerEventFromProto(
+          eventProto.getInputInitializerEvent());
+    } else if (eventProto.hasVmEvent()) {
+      evt = ProtoConverters.convertVertexManagerEventFromProto(
+          eventProto.getVmEvent());
+    } else if (eventProto.hasRootInputDataInformationEvent()) {
+      evt = ProtoConverters.convertRootInputDataInformationEventFromProto(
+          eventProto.getRootInputDataInformationEvent());
+    } else {
+      throw new IOException("Unsupported TezEvent type");
+    }
+
+    EventMetaData sourceInfo = null;
+    EventMetaData destinationInfo = null;
+    if (eventProto.hasSourceInfo()) {
+      sourceInfo = convertEventMetaDataFromProto(eventProto.getSourceInfo());
+    }
+    if (eventProto.hasDestinationInfo()) {
+      destinationInfo = convertEventMetaDataFromProto(eventProto.getDestinationInfo());
+    }
+    TezEvent tezEvent = new TezEvent(evt, sourceInfo, eventProto.getEventTime());
+    tezEvent.setDestinationInfo(destinationInfo);
+    return tezEvent;
+  }
+  
+  public static RecoveryProtos.EventMetaDataProto convertEventMetaDataToProto(
+      EventMetaData eventMetaData) {
+    RecoveryProtos.EventMetaDataProto.Builder builder =
+        RecoveryProtos.EventMetaDataProto.newBuilder()
+        .setProducerConsumerType(eventMetaData.getEventGenerator().ordinal())
+        .setEdgeVertexName(eventMetaData.getEdgeVertexName())
+        .setTaskVertexName(eventMetaData.getTaskVertexName());
+    if (eventMetaData.getTaskAttemptID() != null) {
+        builder.setTaskAttemptId(eventMetaData.getTaskAttemptID().toString());
+    }
+    return builder.build();
+  }
+
+  public static EventMetaData convertEventMetaDataFromProto(
+      RecoveryProtos.EventMetaDataProto proto) {
+    TezTaskAttemptID attemptID = null;
+    if (proto.hasTaskAttemptId()) {
+      attemptID = TezTaskAttemptID.fromString(proto.getTaskAttemptId());
+    }
+    return new EventMetaData(
+        EventMetaData.EventProducerConsumerType.values()[proto.getProducerConsumerType()],
+        proto.getTaskVertexName(),
+        proto.getEdgeVertexName(),
+        attemptID);
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/proto/HistoryEvents.proto
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/proto/HistoryEvents.proto b/tez-dag/src/main/proto/HistoryEvents.proto
index 5392d8a..b9e4507 100644
--- a/tez-dag/src/main/proto/HistoryEvents.proto
+++ b/tez-dag/src/main/proto/HistoryEvents.proto
@@ -90,6 +90,8 @@ message VertexInitializedProto {
   optional int64 init_time = 4;
   optional int32 num_tasks = 5;
   repeated RootInputLeafOutputProto inputs = 6;
+  repeated TezEventProto init_generated_events = 7; 
+  optional bool isReconfigurePlanned = 8;
 }
 
 message VertexStartedProto {
@@ -99,6 +101,16 @@ message VertexStartedProto {
   optional int64 start_time = 4;
 }
 
+message VertexConfigurationDoneProto {
+  optional string vertex_id = 1;
+  optional int64 reconfigure_done_time = 2;
+  optional int32 num_tasks = 3;
+  optional VertexLocationHintProto vertex_location_hint = 4;
+  repeated EdgeManagerDescriptorProto edge_manager_descriptors = 5;
+  repeated RootInputSpecUpdateProto root_input_spec_updates = 6;
+  optional bool setParallelismCalled_flag = 7;
+}
+
 message EdgeManagerDescriptorProto {
   optional string edge_name = 1;
   optional PlanEdgeProperty edge_property = 2;
@@ -110,14 +122,6 @@ message RootInputSpecUpdateProto {
   repeated int32 num_physical_inputs = 3;
 }
 
-message VertexParallelismUpdatedProto {
-  optional string vertex_id = 1;
-  optional int32 num_tasks = 2;
-  optional VertexLocationHintProto vertex_location_hint = 3;
-  repeated EdgeManagerDescriptorProto edge_manager_descriptors = 4;
-  repeated RootInputSpecUpdateProto root_input_spec_updates = 5;
-}
-
 message VertexCommitStartedProto {
   optional string vertex_id = 1;
 }
@@ -129,11 +133,13 @@ message VertexCommitFinishedProto {
 message VertexGroupCommitStartedProto {
   optional string dag_id = 1;
   optional string vertex_group_name = 2;
+  repeated string vertex_ids = 3;
 }
 
 message VertexGroupCommitFinishedProto {
   optional string dag_id = 1;
   optional string vertex_group_name = 2;
+  repeated string vertex_ids = 3;
 }
 
 message VertexFinishedProto {
@@ -183,6 +189,7 @@ message TaskAttemptFinishedProto {
   optional TezCountersProto counters = 9;
   optional string error_enum = 10;
   repeated DataEventDependencyInfoProto data_events = 11;
+  repeated TezEventProto ta_generated_events = 12;
 }
 
 message EventMetaDataProto {
@@ -192,7 +199,7 @@ message EventMetaDataProto {
   optional string task_attempt_id = 4;
 }
 
-message TezDataMovementEventProto {
+message TezEventProto {
   optional EventMetaDataProto source_info = 1;
   optional EventMetaDataProto destination_info = 2;
   optional DataMovementEventProto data_movement_event = 3;
@@ -200,11 +207,7 @@ message TezDataMovementEventProto {
   optional RootInputDataInformationEventProto root_input_data_information_event = 5;
   optional RootInputInitializerEventProto input_initializer_event = 6;
   optional int64 event_time = 7;
-}
-
-message VertexDataMovementEventsGeneratedProto {
-  optional string vertex_id = 1;
-  repeated TezDataMovementEventProto tez_data_movement_event = 2;
+  optional VertexManagerEventProto vm_event = 8;
 }
 
 message SummaryEventProto {

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestVertexStatusBuilder.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestVertexStatusBuilder.java b/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestVertexStatusBuilder.java
index 521ed50..0159372 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestVertexStatusBuilder.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/api/client/TestVertexStatusBuilder.java
@@ -33,12 +33,7 @@ public class TestVertexStatusBuilder {
           VertexStatusBuilder.getProtoState(state);
       VertexStatus.State clientState =
           VertexStatus.getState(stateProto);
-      if (state.equals(VertexState.RECOVERING)) {
-        Assert.assertEquals(clientState.name(),
-            State.NEW.name());
-      } else {
-        Assert.assertEquals(state.name(), clientState.name());
-      }
+      Assert.assertEquals(state.name(), clientState.name());
     }
   }
 


[5/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
index be67cb2..9a45859 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestRecoveryParser.java
@@ -19,8 +19,11 @@
 package org.apache.tez.dag.app;
 
 import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.HashSet;
+import java.util.List;
 import java.util.Map;
 import java.util.Random;
 
@@ -29,11 +32,21 @@ import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.NodeId;
 import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.log4j.Level;
+import org.apache.log4j.LogManager;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
+import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData;
 import org.apache.tez.dag.app.RecoveryParser.DAGSummaryData;
-import org.apache.tez.dag.app.RecoveryParser.RecoveredDAGData;
+import org.apache.tez.dag.app.RecoveryParser.TaskAttemptRecoveryData;
+import org.apache.tez.dag.app.RecoveryParser.TaskRecoveryData;
+import org.apache.tez.dag.app.RecoveryParser.VertexRecoveryData;
 import org.apache.tez.dag.app.dag.DAGState;
+import org.apache.tez.dag.app.dag.VertexState;
 import org.apache.tez.dag.app.dag.impl.DAGImpl;
 import org.apache.tez.dag.app.dag.impl.TestDAGImpl;
 import org.apache.tez.dag.history.DAGHistoryEvent;
@@ -42,10 +55,27 @@ import org.apache.tez.dag.history.events.DAGFinishedEvent;
 import org.apache.tez.dag.history.events.DAGInitializedEvent;
 import org.apache.tez.dag.history.events.DAGStartedEvent;
 import org.apache.tez.dag.history.events.DAGSubmittedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.history.events.TaskFinishedEvent;
+import org.apache.tez.dag.history.events.TaskStartedEvent;
+import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
+import org.apache.tez.dag.history.events.VertexFinishedEvent;
+import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
+import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
+import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.history.recovery.RecoveryService;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.apache.tez.runtime.api.events.DataMovementEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 import org.junit.*;
 
+import com.google.common.collect.Lists;
+
 import static org.junit.Assert.*;
 import static org.mockito.Mockito.*;
 
@@ -75,6 +105,7 @@ public class TestRecoveryParser {
     mockDAGImpl = mock(DAGImpl.class);
     when(mockAppMaster.createDAG(any(DAGPlan.class), any(TezDAGID.class))).thenReturn(mockDAGImpl);
     parser = new RecoveryParser(mockAppMaster, localFS, recoveryPath, 3);
+    LogManager.getRootLogger().setLevel(Level.DEBUG);
   }
 
   private DAGSummaryData createDAGSummaryData(TezDAGID dagId, boolean completed) {
@@ -155,14 +186,14 @@ public class TestRecoveryParser {
         new DAGStartedEvent(dagID, 1L, "user", "dag1")));
     rService.stop();
 
-    RecoveredDAGData dagData = parser.parseRecoveryData();
+    DAGRecoveryData dagData = parser.parseRecoveryData();
     assertEquals(true, dagData.nonRecoverable);
     assertTrue(dagData.reason.contains("DAG Commit was in progress, not recoverable,"));
     // DAGSubmittedEvent is handled but DAGInitializedEvent and DAGStartedEvent in the next attempt are both skipped
     // due to the dag is not recoerable.
     verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class));
-    verify(dagData.recoveredDAG, never()).restoreFromEvent(isA(DAGInitializedEvent.class));
-    verify(dagData.recoveredDAG, never()).restoreFromEvent(isA(DAGStartedEvent.class));
+    assertNull(dagData.getDAGInitializedEvent());
+    assertNull(dagData.getDAGStartedEvent());
   }
 
   // skipAllOtherEvents due to dag finished
@@ -202,7 +233,7 @@ public class TestRecoveryParser {
        new DAGStartedEvent(dagID, 1L, "user", "dag1")));
     rService.stop();
 
-    RecoveredDAGData dagData = parser.parseRecoveryData();
+    DAGRecoveryData dagData = parser.parseRecoveryData();
     assertEquals(false, dagData.nonRecoverable);
     assertEquals(DAGState.FAILED, dagData.dagState);
     assertEquals(true, dagData.isCompleted);
@@ -210,9 +241,8 @@ public class TestRecoveryParser {
     verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class));
     // DAGInitializedEvent may not been handled before DAGFinishedEvent,
     // because DAGFinishedEvent's writeToRecoveryImmediately is true
-    verify(dagData.recoveredDAG).restoreFromEvent(isA(DAGFinishedEvent.class));
-    // DAGStartedEvent is skipped due to it is after DAGFinishedEvent
-    verify(dagData.recoveredDAG, never()).restoreFromEvent(isA(DAGStartedEvent.class));
+    assertNotNull(dagData.getDAGFinishedEvent());
+    assertNull(dagData.getDAGStartedEvent());
   }
 
   @Test(timeout = 5000)
@@ -250,13 +280,13 @@ public class TestRecoveryParser {
     rService.stop();
 
     // corrupted last records will be skipped but the whole recovery logs will be read
-    RecoveredDAGData dagData = parser.parseRecoveryData();
+    DAGRecoveryData dagData = parser.parseRecoveryData();
     assertEquals(false, dagData.isCompleted);
     assertEquals(null, dagData.reason);
     assertEquals(false, dagData.nonRecoverable);
     // verify DAGSubmitedEvent & DAGInititlizedEvent is handled.
     verify(mockAppMaster).createDAG(any(DAGPlan.class),any(TezDAGID.class));
-    verify(dagData.recoveredDAG).restoreFromEvent(isA(DAGInitializedEvent.class));
+    assertNotNull(dagData.getDAGInitializedEvent());
   }
 
   @Test(timeout = 5000)
@@ -293,4 +323,434 @@ public class TestRecoveryParser {
     }
   }
 
+  @Test(timeout=5000)
+  public void testRecoverableSummary_DAGInCommitting() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    RecoveryService rService = new RecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGCommitStartedEvent(dagID, 0L)));
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertEquals(dagID, dagData.recoveredDagID);
+    assertTrue(dagData.nonRecoverable);
+    assertTrue(dagData.reason.contains("DAG Commit was in progress"));
+  }
+  
+  @Test(timeout=5000)
+  public void testRecoverableSummary_DAGFinishCommitting() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    RecoveryService rService = new RecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGCommitStartedEvent(dagID, 0L)));
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGFinishedEvent(dagID, 1L, 2L, DAGState.FAILED, "diag", null, "user", "dag1", null,
+            appAttemptId)));
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertEquals(dagID, dagData.recoveredDagID);
+    assertEquals(DAGState.FAILED, dagData.dagState);
+    assertFalse(dagData.nonRecoverable);
+    assertNull(dagData.reason);
+    assertTrue(dagData.isCompleted);
+  }
+
+  @Test(timeout=5000)
+  public void testRecoverableSummary_VertexInCommitting() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    RecoveryService rService = new RecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexCommitStartedEvent(TezVertexID.getInstance(dagID, 0), 0L)));
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertEquals(dagID, dagData.recoveredDagID);
+    assertTrue(dagData.nonRecoverable);
+    assertTrue(dagData.reason.contains("Vertex Commit was in progress"));
+  }
+
+  @Test(timeout=5000)
+  public void testRecoverableSummary_VertexFinishCommitting() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    RecoveryService rService = new RecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    TezVertexID vertexId = TezVertexID.getInstance(dagID, 0);
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexCommitStartedEvent(vertexId, 0L)));
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexFinishedEvent(vertexId, "v1", 10, 0L, 0L, 
+            0L, 0L, 0L, VertexState.SUCCEEDED, 
+            "", null, null, null)));
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertEquals(dagID, dagData.recoveredDagID);
+    assertFalse(dagData.nonRecoverable);
+  }
+
+  @Test(timeout=5000)
+  public void testRecoverableSummary_VertexGroupInCommitting() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    RecoveryService rService = new RecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexGroupCommitStartedEvent(dagID, "group_1", 
+            Lists.newArrayList(TezVertexID.getInstance(dagID, 0), TezVertexID.getInstance(dagID, 1)), 0L)));
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertEquals(dagID, dagData.recoveredDagID);
+    assertTrue(dagData.nonRecoverable);
+    assertTrue(dagData.reason.contains("Vertex Group Commit was in progress"));
+  }
+  
+  @Test(timeout=5000)
+  public void testRecoverableSummary_VertexGroupFinishCommitting() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    RecoveryService rService = new RecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    TezVertexID v0 = TezVertexID.getInstance(dagID, 0);
+    TezVertexID v1 = TezVertexID.getInstance(dagID, 1);
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexGroupCommitStartedEvent(dagID, "group_1", 
+            Lists.newArrayList(v0, v1), 0L)));
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexGroupCommitFinishedEvent(dagID, "group_1", 
+            Lists.newArrayList(v0, v1), 0L)));
+    // also write VertexFinishedEvent, otherwise it is still non-recoverable
+    // when checking with non-summary event
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexFinishedEvent(v0, "v1", 10, 0L, 0L, 
+            0L, 0L, 0L, VertexState.SUCCEEDED, 
+            "", null, null, null)));
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexFinishedEvent(v1, "v1", 10, 0L, 0L, 
+            0L, 0L, 0L, VertexState.SUCCEEDED, 
+            "", null, null, null)));
+    rService.stop();
+    
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertEquals(dagID, dagData.recoveredDagID);
+    assertFalse(dagData.nonRecoverable);
+  }
+  
+  @Test(timeout=5000)
+  public void testRecoverableNonSummary1() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    // MockRecoveryService will skip the non-summary event
+    MockRecoveryService rService = new MockRecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    TezVertexID vertexId = TezVertexID.getInstance(dagID, 0);
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexCommitStartedEvent(vertexId, 0L)));
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexFinishedEvent(vertexId, "v1", 10, 0L, 0L, 
+            0L, 0L, 0L, VertexState.SUCCEEDED, 
+            "", null, null, null)));
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertTrue(dagData.nonRecoverable);
+    assertTrue(dagData.reason.contains("Vertex has been committed, but its full recovery events are not seen"));
+  }
+  
+  @Test(timeout=5000)
+  public void testRecoverableNonSummary2() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    // MockRecoveryService will skip the non-summary event
+    MockRecoveryService rService = new MockRecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // write a DAGSubmittedEvent first to initialize summaryStream
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    // It should be fine to skip other events, just for testing.
+    TezVertexID vertexId = TezVertexID.getInstance(dagID, 0);
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexGroupCommitStartedEvent(dagID, "group_1", 
+            Lists.newArrayList(TezVertexID.getInstance(dagID, 0), TezVertexID.getInstance(dagID, 1)), 0L)));
+    rService.handle(new DAGHistoryEvent(dagID,
+        new VertexGroupCommitFinishedEvent(dagID, "group_1", 
+            Lists.newArrayList(TezVertexID.getInstance(dagID, 0), TezVertexID.getInstance(dagID, 1)), 0L)));
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertTrue(dagData.nonRecoverable);
+    assertTrue(dagData.reason.contains("Vertex has been committed as member of vertex group"
+              + ", but its full recovery events are not seen"));
+  }
+
+  @Test(timeout=5000)
+  public void testRecoveryData() throws IOException {
+    ApplicationId appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
+    TezDAGID dagID = TezDAGID.getInstance(appId, 1);
+    ApplicationAttemptId appAttemptId = ApplicationAttemptId.newInstance(appId, 1);
+    AppContext appContext = mock(AppContext.class);
+    when(appContext.getCurrentRecoveryDir()).thenReturn(new Path(recoveryPath+"/1"));
+    when(appContext.getClock()).thenReturn(new SystemClock());
+    when(mockDAGImpl.getID()).thenReturn(dagID);
+
+    RecoveryService rService = new RecoveryService(appContext);
+    Configuration conf = new Configuration();
+    conf.setBoolean(RecoveryService.TEZ_TEST_RECOVERY_DRAIN_EVENTS_WHEN_STOPPED, true);
+    rService.init(conf);
+    rService.start();
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    // DAG  DAGSubmittedEvent -> DAGInitializedEvent -> DAGStartedEvent
+    rService.handle(new DAGHistoryEvent(dagID,
+        new DAGSubmittedEvent(dagID, 1L, dagPlan, ApplicationAttemptId.newInstance(appId, 1),
+            null, "user", new Configuration(), null)));
+    DAGInitializedEvent dagInitedEvent = new DAGInitializedEvent(dagID, 100L, 
+        "user", "dagName", null);
+    DAGStartedEvent dagStartedEvent = new DAGStartedEvent(dagID, 0L, "user", "dagName");
+    rService.handle(new DAGHistoryEvent(dagID, dagInitedEvent));
+    rService.handle(new DAGHistoryEvent(dagID, dagStartedEvent));
+    
+    // 3 vertices of this dag: v0, v1, v2
+    TezVertexID v0Id = TezVertexID.getInstance(dagID, 0);
+    TezVertexID v1Id = TezVertexID.getInstance(dagID, 1);
+    TezVertexID v2Id = TezVertexID.getInstance(dagID, 2);
+    // v0 VertexInitializedEvent
+    VertexInitializedEvent v0InitedEvent =  new VertexInitializedEvent(v0Id, "v0", 200L, 400L, 2, null, null, null);
+    rService.handle(new DAGHistoryEvent(dagID, v0InitedEvent));
+    // v1 VertexFinishedEvent(KILLED)
+    VertexFinishedEvent v1FinishedEvent = new VertexFinishedEvent(v1Id, "v1", 2, 300L, 400L, 
+        500L, 600L, 700L, VertexState.KILLED, 
+        "", null, null, null);
+    rService.handle(new DAGHistoryEvent(dagID, v1FinishedEvent));
+    // v2 VertexInitializedEvent -> VertexStartedEvent
+    List<TezEvent> initGeneratedEvents = Lists.newArrayList(
+        new TezEvent(DataMovementEvent.create(ByteBuffer.wrap(new byte[0])), null));
+    VertexInitializedEvent v2InitedEvent = new VertexInitializedEvent(v2Id, "v2", 200L, 300L,
+        2, null, null, initGeneratedEvents);
+    VertexStartedEvent v2StartedEvent = new VertexStartedEvent(v2Id, 0L, 0L);
+    rService.handle(new DAGHistoryEvent(dagID, v2InitedEvent));
+    rService.handle(new DAGHistoryEvent(dagID, v2StartedEvent));
+
+    // 3 tasks of v2
+    TezTaskID t0v2Id = TezTaskID.getInstance(v2Id, 0);
+    TezTaskID t1v2Id = TezTaskID.getInstance(v2Id, 1);
+    TezTaskID t2v2Id = TezTaskID.getInstance(v2Id, 2);
+    // t0v2 TaskStartedEvent
+    TaskStartedEvent t0v2StartedEvent = new TaskStartedEvent(t0v2Id, "v2", 400L, 5000L);
+    rService.handle(new DAGHistoryEvent(dagID, t0v2StartedEvent));
+    // t1v2 TaskFinishedEvent
+    TaskFinishedEvent t1v2FinishedEvent = new TaskFinishedEvent(t1v2Id, "v1",
+        0L, 0L, null, TaskState.KILLED, "", null, 4);
+    rService.handle(new DAGHistoryEvent(dagID, t1v2FinishedEvent));
+    // t2v2 TaskStartedEvent -> TaskFinishedEvent
+    TaskStartedEvent t2v2StartedEvent = new TaskStartedEvent(t2v2Id, "v2", 400L, 500L);
+    rService.handle(new DAGHistoryEvent(dagID, t2v2StartedEvent));
+    TaskFinishedEvent t2v2FinishedEvent = new TaskFinishedEvent(t2v2Id, "v1",
+        0L, 0L, null, TaskState.SUCCEEDED, "", null, 4);
+    rService.handle(new DAGHistoryEvent(dagID, t2v2FinishedEvent));
+
+    // attempts under t0v2
+    ContainerId containerId = ContainerId.newContainerId(appAttemptId, 1);
+    NodeId nodeId = NodeId.newInstance("localhost", 9999);
+    TezTaskAttemptID ta0t0v2Id = TezTaskAttemptID.getInstance(t0v2Id, 0);
+    TaskAttemptStartedEvent ta0t0v2StartedEvent = new TaskAttemptStartedEvent(
+        ta0t0v2Id, "v1", 0L, containerId, 
+        nodeId, "", "", "");
+    rService.handle(new DAGHistoryEvent(dagID, ta0t0v2StartedEvent));
+    // attempts under t2v2
+    TezTaskAttemptID ta0t2v2Id = TezTaskAttemptID.getInstance(t2v2Id, 0);
+    TaskAttemptStartedEvent ta0t2v2StartedEvent = new TaskAttemptStartedEvent(
+        ta0t2v2Id, "v1", 500L, containerId, 
+        nodeId, "", "", "");
+    rService.handle(new DAGHistoryEvent(dagID, ta0t2v2StartedEvent));
+    TaskAttemptFinishedEvent ta0t2v2FinishedEvent = new TaskAttemptFinishedEvent(
+        ta0t2v2Id, "v1", 500L, 600L, 
+        TaskAttemptState.SUCCEEDED, null, "", null, 
+        null, null, 0L, null, 0L);
+    rService.handle(new DAGHistoryEvent(dagID, ta0t2v2FinishedEvent));
+
+    rService.stop();
+
+    DAGRecoveryData dagData = parser.parseRecoveryData();
+    assertFalse(dagData.nonRecoverable);
+    // There's no equals method for the history event, so here only verify the init/start/finish time of each event for simplicity
+    assertEquals(dagInitedEvent.getInitTime(), dagData.getDAGInitializedEvent().getInitTime());
+    assertEquals(dagStartedEvent.getStartTime(), dagData.getDAGStartedEvent().getStartTime());
+    assertNull(dagData.getDAGFinishedEvent());
+
+    VertexRecoveryData v0Data = dagData.getVertexRecoveryData(v0Id);
+    VertexRecoveryData v1Data = dagData.getVertexRecoveryData(v1Id);
+    VertexRecoveryData v2Data = dagData.getVertexRecoveryData(v2Id);
+    assertNotNull(v0Data);
+    assertNotNull(v1Data);
+    assertNotNull(v2Data);
+    assertEquals(v0InitedEvent.getInitedTime(), v0Data.getVertexInitedEvent().getInitedTime());
+    assertNull(v0Data.getVertexStartedEvent());
+    assertNull(v1Data.getVertexInitedEvent());
+    assertEquals(v1FinishedEvent.getFinishTime(), v1Data.getVertexFinishedEvent().getFinishTime());
+    assertEquals(v2InitedEvent.getInitedTime(), v2Data.getVertexInitedEvent().getInitedTime());
+    assertEquals(v2StartedEvent.getStartTime(), v2Data.getVertexStartedEvent().getStartTime());
+
+    TaskRecoveryData t0v2Data = dagData.getTaskRecoveryData(t0v2Id);
+    TaskRecoveryData t1v2Data = dagData.getTaskRecoveryData(t1v2Id);
+    TaskRecoveryData t2v2Data = dagData.getTaskRecoveryData(t2v2Id);
+    assertNotNull(t0v2Data);
+    assertNotNull(t1v2Data);
+    assertNotNull(t2v2Data);
+    assertEquals(t0v2StartedEvent.getStartTime(), t0v2Data.getTaskStartedEvent().getStartTime());
+    assertNull(t0v2Data.getTaskFinishedEvent());
+    assertEquals(t1v2FinishedEvent.getFinishTime(), t1v2Data.getTaskFinishedEvent().getFinishTime());
+    assertNull(t1v2Data.getTaskStartedEvent());
+    assertEquals(t2v2StartedEvent.getStartTime(), t2v2Data.getTaskStartedEvent().getStartTime());
+    assertEquals(t2v2FinishedEvent.getFinishTime(), t2v2Data.getTaskFinishedEvent().getFinishTime());
+
+    TaskAttemptRecoveryData ta0t0v2Data = dagData.getTaskAttemptRecoveryData(ta0t0v2Id);
+    TaskAttemptRecoveryData ta0t2v2Data = dagData.getTaskAttemptRecoveryData(ta0t2v2Id);
+    assertNotNull(ta0t0v2Data);
+    assertNotNull(ta0t2v2Data);
+    assertEquals(ta0t0v2StartedEvent.getStartTime(), ta0t0v2Data.getTaskAttemptStartedEvent().getStartTime());
+    assertNull(ta0t0v2Data.getTaskAttemptFinishedEvent());
+    assertEquals(ta0t2v2StartedEvent.getStartTime(), ta0t2v2Data.getTaskAttemptStartedEvent().getStartTime());
+    assertEquals(ta0t2v2FinishedEvent.getFinishTime(), ta0t2v2Data.getTaskAttemptFinishedEvent().getFinishTime());
+  }
+
+  // Simulate the behavior that summary event is written 
+  // but non-summary is not written to hdfs
+  public static class MockRecoveryService extends RecoveryService{
+
+    public MockRecoveryService(AppContext appContext) {
+      super(appContext);
+    }
+
+    @Override
+    protected void handleRecoveryEvent(DAGHistoryEvent event)
+        throws IOException {
+      // skip the non-summary events
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java
index 17fa4d9..3f80928 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestTaskCommunicatorManager1.java
@@ -35,6 +35,7 @@ import java.util.Map;
 import java.util.Random;
 
 import com.google.common.collect.Lists;
+
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.Text;
 import org.apache.hadoop.security.Credentials;
@@ -68,7 +69,9 @@ import org.apache.tez.common.TezTaskUmbilicalProtocol;
 import org.apache.tez.dag.api.TaskCommunicatorContext;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventTezEventUpdate;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
@@ -83,6 +86,8 @@ import org.apache.tez.runtime.api.events.DataMovementEvent;
 import org.apache.tez.runtime.api.events.InputReadErrorEvent;
 import org.apache.tez.runtime.api.events.TaskAttemptCompletedEvent;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 import org.apache.tez.runtime.api.impl.EventType;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.api.impl.TezEvent;
@@ -235,15 +240,18 @@ public class TestTaskCommunicatorManager1 {
   @Test (timeout = 5000)
   public void testTaskEventRouting() throws Exception {
     List<TezEvent> events =  Arrays.asList(
-      new TezEvent(new TaskStatusUpdateEvent(null, 0.0f, null, false), null),
-      new TezEvent(DataMovementEvent.create(0, ByteBuffer.wrap(new byte[0])), null),
-      new TezEvent(new TaskAttemptCompletedEvent(), null)
+      new TezEvent(new TaskStatusUpdateEvent(null, 0.0f, null, false), new EventMetaData(EventProducerConsumerType.PROCESSOR,
+          "v1", "v2", taskAttemptID)),
+      new TezEvent(DataMovementEvent.create(0, ByteBuffer.wrap(new byte[0])), new EventMetaData(EventProducerConsumerType.OUTPUT,
+          "v1", "v2", taskAttemptID)),
+      new TezEvent(new TaskAttemptCompletedEvent(), new EventMetaData(EventProducerConsumerType.SYSTEM,
+          "v1", "v2", taskAttemptID))
     );
 
     generateHeartbeat(events, 0, 1, 0, new ArrayList<TezEvent>());
 
     ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
-    verify(eventHandler, times(2)).handle(arg.capture());
+    verify(eventHandler, times(4)).handle(arg.capture());
     final List<Event> argAllValues = arg.getAllValues();
 
     final Event statusUpdateEvent = argAllValues.get(0);
@@ -251,28 +259,33 @@ public class TestTaskCommunicatorManager1 {
         statusUpdateEvent.getType());
     assertEquals(false, ((TaskAttemptEventStatusUpdate)statusUpdateEvent).getReadErrorReported());
 
-    final Event vertexEvent = argAllValues.get(1);
-    final VertexEventRouteEvent vertexRouteEvent = (VertexEventRouteEvent)vertexEvent;
-    assertEquals("First event should be routed to vertex", VertexEventType.V_ROUTE_EVENT,
-        vertexEvent.getType());
+    final TaskAttemptEventTezEventUpdate taEvent = (TaskAttemptEventTezEventUpdate)argAllValues.get(1);
+    assertEquals(1, taEvent.getTezEvents().size());
+    assertEquals(EventType.DATA_MOVEMENT_EVENT,
+        taEvent.getTezEvents().get(0).getEventType());
+    
+    final TaskAttemptEvent taCompleteEvent = (TaskAttemptEvent)argAllValues.get(2);
+    assertEquals(TaskAttemptEventType.TA_DONE, taCompleteEvent.getType());
+    final VertexEventRouteEvent vertexRouteEvent = (VertexEventRouteEvent)argAllValues.get(3);
+    assertEquals(1, vertexRouteEvent.getEvents().size());
     assertEquals(EventType.DATA_MOVEMENT_EVENT,
         vertexRouteEvent.getEvents().get(0).getEventType());
-    assertEquals(EventType.TASK_ATTEMPT_COMPLETED_EVENT,
-        vertexRouteEvent.getEvents().get(1).getEventType());
   }
   
   @Test (timeout = 5000)
   public void testTaskEventRoutingWithReadError() throws Exception {
     List<TezEvent> events =  Arrays.asList(
       new TezEvent(new TaskStatusUpdateEvent(null, 0.0f, null, false), null),
-      new TezEvent(InputReadErrorEvent.create("", 0, 0), null),
-      new TezEvent(new TaskAttemptCompletedEvent(), null)
+      new TezEvent(InputReadErrorEvent.create("", 0, 0), new EventMetaData(EventProducerConsumerType.INPUT,
+          "v2", "v1", taskAttemptID)),
+      new TezEvent(new TaskAttemptCompletedEvent(), new EventMetaData(EventProducerConsumerType.SYSTEM,
+          "v1", "v2", taskAttemptID))
     );
 
     generateHeartbeat(events, 0, 1, 0, new ArrayList<TezEvent>());
 
     ArgumentCaptor<Event> arg = ArgumentCaptor.forClass(Event.class);
-    verify(eventHandler, times(2)).handle(arg.capture());
+    verify(eventHandler, times(3)).handle(arg.capture());
     final List<Event> argAllValues = arg.getAllValues();
 
     final Event statusUpdateEvent = argAllValues.get(0);
@@ -280,22 +293,24 @@ public class TestTaskCommunicatorManager1 {
         statusUpdateEvent.getType());
     assertEquals(true, ((TaskAttemptEventStatusUpdate)statusUpdateEvent).getReadErrorReported());
 
-    final Event vertexEvent = argAllValues.get(1);
+    final Event taFinishedEvent = argAllValues.get(1);
+    assertEquals("Second event should be TA_DONE", TaskAttemptEventType.TA_DONE,
+        taFinishedEvent.getType());
+
+    final Event vertexEvent = argAllValues.get(2);
     final VertexEventRouteEvent vertexRouteEvent = (VertexEventRouteEvent)vertexEvent;
-    assertEquals("First event should be routed to vertex", VertexEventType.V_ROUTE_EVENT,
+    assertEquals("Third event should be routed to vertex", VertexEventType.V_ROUTE_EVENT,
         vertexEvent.getType());
     assertEquals(EventType.INPUT_READ_ERROR_EVENT,
         vertexRouteEvent.getEvents().get(0).getEventType());
-    assertEquals(EventType.TASK_ATTEMPT_COMPLETED_EVENT,
-        vertexRouteEvent.getEvents().get(1).getEventType());
-
   }
 
 
   @Test (timeout = 5000)
   public void testTaskEventRoutingTaskAttemptOnly() throws Exception {
     List<TezEvent> events = Arrays.asList(
-      new TezEvent(new TaskAttemptCompletedEvent(), null)
+      new TezEvent(new TaskAttemptCompletedEvent(), new EventMetaData(EventProducerConsumerType.SYSTEM,
+          "v1", "v2", taskAttemptID))
     );
     generateHeartbeat(events, 0, 1, 0, new ArrayList<TezEvent>());
 
@@ -304,7 +319,8 @@ public class TestTaskCommunicatorManager1 {
     final List<Event> argAllValues = arg.getAllValues();
 
     final Event event = argAllValues.get(0);
-    assertEquals("only event should be route event", VertexEventType.V_ROUTE_EVENT,
+    // Route to TaskAttempt directly rather than through Vertex
+    assertEquals("only event should be route event", TaskAttemptEventType.TA_DONE,
         event.getType());
   }
   


[7/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
index 5b75179..758c637 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexImpl.java
@@ -17,7 +17,10 @@
 
 package org.apache.tez.dag.app.dag.impl;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.nio.ByteBuffer;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
 import java.util.BitSet;
@@ -41,8 +44,6 @@ import java.util.concurrent.locks.ReentrantReadWriteLock;
 
 import javax.annotation.Nullable;
 
-import com.google.common.base.Strings;
-
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.lang.exception.ExceptionUtils;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
@@ -64,8 +65,8 @@ import org.apache.tez.common.ReflectionUtils;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.EdgeManagerPluginDescriptor;
-import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
 import org.apache.tez.dag.api.EdgeProperty;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.InputInitializerDescriptor;
 import org.apache.tez.dag.api.OutputCommitterDescriptor;
@@ -73,12 +74,15 @@ import org.apache.tez.dag.api.OutputDescriptor;
 import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.RootInputLeafOutput;
 import org.apache.tez.dag.api.Scope;
+import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
 import org.apache.tez.dag.api.TezException;
 import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.api.VertexLocationHint;
-import org.apache.tez.dag.api.TaskLocationHint;
+import org.apache.tez.dag.api.VertexManagerPlugin;
+import org.apache.tez.dag.api.VertexManagerPluginContext;
 import org.apache.tez.dag.api.VertexManagerPluginContext.ScheduleTaskRequest;
 import org.apache.tez.dag.api.VertexManagerPluginDescriptor;
 import org.apache.tez.dag.api.client.ProgressBuilder;
@@ -94,6 +98,7 @@ import org.apache.tez.dag.api.records.DAGProtos.RootInputLeafOutputProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.RecoveryParser.VertexRecoveryData;
 import org.apache.tez.dag.app.TaskAttemptEventInfo;
 import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
@@ -113,10 +118,7 @@ import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.DAGEventVertexCompleted;
 import org.apache.tez.dag.app.dag.event.DAGEventVertexReRunning;
 import org.apache.tez.dag.app.dag.event.SpeculatorEvent;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.TaskEvent;
-import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
 import org.apache.tez.dag.app.dag.event.TaskEventScheduleTask;
 import org.apache.tez.dag.app.dag.event.TaskEventTermination;
 import org.apache.tez.dag.app.dag.event.TaskEventType;
@@ -130,24 +132,20 @@ import org.apache.tez.dag.app.dag.event.VertexEventRootInputFailed;
 import org.apache.tez.dag.app.dag.event.VertexEventRootInputInitialized;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.dag.event.VertexEventSourceTaskAttemptCompleted;
-import org.apache.tez.dag.app.dag.event.VertexEventSourceVertexRecovered;
 import org.apache.tez.dag.app.dag.event.VertexEventSourceVertexStarted;
 import org.apache.tez.dag.app.dag.event.VertexEventTaskAttemptCompleted;
 import org.apache.tez.dag.app.dag.event.VertexEventTaskCompleted;
 import org.apache.tez.dag.app.dag.event.VertexEventTaskReschedule;
 import org.apache.tez.dag.app.dag.event.VertexEventTermination;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.impl.DAGImpl.VertexGroupInfo;
 import org.apache.tez.dag.app.dag.impl.Edge.PendingEventRouteMetadata;
 import org.apache.tez.dag.app.dag.speculation.legacy.LegacySpeculator;
 import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
 import org.apache.tez.dag.history.events.VertexFinishedEvent;
 import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
 import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.library.vertexmanager.InputReadyVertexManager;
 import org.apache.tez.dag.library.vertexmanager.ShuffleVertexManager;
@@ -158,19 +156,19 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.InputSpecUpdate;
 import org.apache.tez.runtime.api.InputStatistics;
 import org.apache.tez.runtime.api.OutputCommitter;
 import org.apache.tez.runtime.api.OutputCommitterContext;
-import org.apache.tez.runtime.api.InputSpecUpdate;
 import org.apache.tez.runtime.api.OutputStatistics;
 import org.apache.tez.runtime.api.TaskAttemptIdentifier;
 import org.apache.tez.runtime.api.VertexStatistics;
 import org.apache.tez.runtime.api.events.CompositeDataMovementEvent;
 import org.apache.tez.runtime.api.events.DataMovementEvent;
-import org.apache.tez.runtime.api.events.InputFailedEvent;
 import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.events.InputFailedEvent;
 import org.apache.tez.runtime.api.events.InputInitializerEvent;
-import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
 import org.apache.tez.runtime.api.events.VertexManagerEvent;
 import org.apache.tez.runtime.api.impl.EventMetaData;
 import org.apache.tez.runtime.api.impl.EventType;
@@ -180,9 +178,14 @@ import org.apache.tez.runtime.api.impl.OutputSpec;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 import org.apache.tez.runtime.api.impl.TaskStatistics;
 import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.state.OnStateChangedCallback;
+import org.apache.tez.state.StateMachineTez;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
+import com.google.common.base.Strings;
 import com.google.common.collect.HashMultiset;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
@@ -192,11 +195,6 @@ import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
 import com.google.common.util.concurrent.ListenableFuture;
 
-import org.apache.tez.state.OnStateChangedCallback;
-import org.apache.tez.state.StateMachineTez;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 /** Implementation of Vertex interface. Maintains the state machines of Vertex.
  * The read and write calls use ReadWriteLock for concurrency.
  */
@@ -222,6 +220,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
   //private final MRAppMetrics metrics;
   private final AppContext appContext;
   private final DAG dag;
+  private final VertexRecoveryData recoveryData;
+  private List<TezEvent> initGeneratedEvents = new ArrayList<TezEvent>();
+  // set it to be true when setParallelism is called(used for recovery) 
+  private boolean setParallelismCalledFlag = false;
 
   private boolean lazyTasksCopyNeeded = false;
   // must be a linked map for ordering
@@ -281,11 +283,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
   private static final VertexStateChangedCallback STATE_CHANGED_CALLBACK =
       new VertexStateChangedCallback();
 
-  private VertexState recoveredState = VertexState.NEW;
-  @VisibleForTesting
-  List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
-  private boolean vertexAlreadyInitialized = false;
-
   @VisibleForTesting
   final List<TezEvent> pendingInitializerEvents = new LinkedList<TezEvent>();
 
@@ -307,7 +304,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
           .addTransition
               (VertexState.NEW,
                   EnumSet.of(VertexState.NEW, VertexState.INITED,
-                      VertexState.INITIALIZING, VertexState.FAILED),
+                      VertexState.INITIALIZING, VertexState.FAILED, VertexState.KILLED),
                   VertexEventType.V_INIT,
                   new InitTransition())
           .addTransition(VertexState.NEW,
@@ -324,21 +321,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
                 SOURCE_TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION)
           .addTransition
               (VertexState.NEW,
-                  EnumSet.of(VertexState.NEW, VertexState.INITED,
-                      VertexState.INITIALIZING, VertexState.RUNNING,
+                  EnumSet.of(VertexState.NEW,
                       VertexState.SUCCEEDED, VertexState.FAILED,
-                      VertexState.KILLED, VertexState.ERROR,
-                      VertexState.RECOVERING),
+                      VertexState.KILLED, VertexState.ERROR),
                   VertexEventType.V_RECOVER,
-                  new StartRecoverTransition())
-          .addTransition
-              (VertexState.NEW,
-                  EnumSet.of(VertexState.NEW, VertexState.INITED,
-                      VertexState.INITIALIZING, VertexState.RUNNING,
-                      VertexState.SUCCEEDED, VertexState.FAILED,
-                      VertexState.KILLED, VertexState.ERROR,
-                      VertexState.RECOVERING),
-                  VertexEventType.V_SOURCE_VERTEX_RECOVERED,
                   new RecoverTransition())
           .addTransition(VertexState.NEW, VertexState.NEW,
               VertexEventType.V_SOURCE_VERTEX_STARTED,
@@ -349,31 +335,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
           .addTransition(VertexState.NEW, VertexState.ERROR,
               VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
-          .addTransition
-              (VertexState.RECOVERING,
-                  EnumSet.of(VertexState.NEW, VertexState.INITED,
-                      VertexState.INITIALIZING, VertexState.RUNNING,
-                      VertexState.SUCCEEDED, VertexState.FAILED,
-                      VertexState.KILLED, VertexState.ERROR,
-                      VertexState.RECOVERING),
-                  VertexEventType.V_SOURCE_VERTEX_RECOVERED,
-                  new RecoverTransition())
-          .addTransition
-              (VertexState.RECOVERING, VertexState.RECOVERING,
-                  EnumSet.of(VertexEventType.V_INIT,
-                      VertexEventType.V_ROUTE_EVENT,
-                      VertexEventType.V_SOURCE_VERTEX_STARTED,
-                      VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED),
-                  new BufferDataRecoverTransition())
-          .addTransition
-              (VertexState.RECOVERING, VertexState.RECOVERING,
-                  VertexEventType.V_TERMINATE,
-                  new TerminateDuringRecoverTransition())
-          .addTransition
-              (VertexState.RECOVERING, EnumSet.of(VertexState.RECOVERING),
-                  VertexEventType.V_MANAGER_USER_CODE_ERROR,
-                  new VertexManagerUserCodeErrorTransition())
-          
+
           // Transitions from INITIALIZING state
           .addTransition(VertexState.INITIALIZING,
               EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
@@ -429,11 +391,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
               EnumSet.of(VertexState.FAILED),
               VertexEventType.V_ROOT_INPUT_FAILED,
               new RootInputInitFailedTransition())
-          .addTransition
-              (VertexState.INITED,
-                  EnumSet.of(VertexState.INITED, VertexState.ERROR),
-                  VertexEventType.V_INIT,
-                  new IgnoreInitInInitedTransition())
           .addTransition(VertexState.INITED, VertexState.INITED,
               VertexEventType.V_SOURCE_VERTEX_STARTED,
               new SourceVertexStartedTransition())
@@ -618,8 +575,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
                   VertexEventType.V_ROOT_INPUT_INITIALIZED,
                   VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED,
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
-                  VertexEventType.V_INPUT_DATA_INFORMATION,
-                  VertexEventType.V_SOURCE_VERTEX_RECOVERED))
+                  VertexEventType.V_INPUT_DATA_INFORMATION))
 
           // Transitions from KILLED state
           .addTransition(
@@ -641,8 +597,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
                   VertexEventType.V_TASK_COMPLETED,
                   VertexEventType.V_ROOT_INPUT_INITIALIZED,
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
-                  VertexEventType.V_INPUT_DATA_INFORMATION,
-                  VertexEventType.V_SOURCE_VERTEX_RECOVERED))
+                  VertexEventType.V_INPUT_DATA_INFORMATION))
 
           // No transitions from INTERNAL_ERROR state. Ignore all.
           .addTransition(
@@ -662,8 +617,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
                   VertexEventType.V_INTERNAL_ERROR,
                   VertexEventType.V_ROOT_INPUT_INITIALIZED,
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
-                  VertexEventType.V_INPUT_DATA_INFORMATION,
-                  VertexEventType.V_SOURCE_VERTEX_RECOVERED))
+                  VertexEventType.V_INPUT_DATA_INFORMATION))
           // create the topology tables
           .installTopology();
 
@@ -676,7 +630,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         .registerStateEnteredCallback(VertexState.KILLED,
             STATE_CHANGED_CALLBACK)
         .registerStateEnteredCallback(VertexState.RUNNING,
-            STATE_CHANGED_CALLBACK);
+            STATE_CHANGED_CALLBACK)
+        .registerStateEnteredCallback(VertexState.INITIALIZING,
+            STATE_CHANGED_CALLBACK);;
   }
 
   private final StateMachineTez<VertexState, VertexEventType, VertexEvent, VertexImpl> stateMachine;
@@ -785,18 +741,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
   private VertexTerminationCause terminationCause;
 
   private String logIdentifier;
-  @VisibleForTesting
-  boolean recoveryCommitInProgress = false;
-  private boolean summaryCompleteSeen = false;
-  @VisibleForTesting
-  boolean hasCommitter = false;
-  private boolean vertexCompleteSeen = false;
-  private Map<String,EdgeProperty> recoveredSourceEdgeProperties = null;
-  private Map<String, InputSpecUpdate> recoveredRootInputSpecUpdates = null;
-
-  // Recovery related flags
-  boolean recoveryInitEventSeen = false;
-  boolean recoveryStartEventSeen = false;
+
   private VertexStats vertexStats = null;
 
   private final TaskSpecificLaunchCmdOption taskSpecificLaunchCmdOpts;
@@ -945,7 +890,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         .createEnvironmentMapFromDAGPlan(vertexPlan.getTaskConfig()
             .getEnvironmentSettingList());
     this.taskSpecificLaunchCmdOpts = taskSpecificLaunchCmdOption;
-
+    this.recoveryData = appContext.getDAGRecoveryData() == null ?
+        null : appContext.getDAGRecoveryData().getVertexRecoveryData(vertexId);
     // Set up log properties, including task specific log properties.
     String javaOptsWithoutLoggerMods =
         vertexPlan.getTaskConfig().hasJavaOpts() ? vertexPlan.getTaskConfig().getJavaOpts() : null;
@@ -1440,110 +1386,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     return this.appContext;
   }
 
-  private void handleParallelismUpdate(int newParallelism,
-      Map<String, EdgeProperty> sourceEdgeProperties,
-      Map<String, InputSpecUpdate> rootInputSpecUpdates, int oldParallelism) {
-    // initial parallelism must have been set by this time
-    // parallelism update is recorded in history only for change from an initialized value
-    Preconditions.checkArgument(oldParallelism != -1, getLogIdentifier());
-    if (oldParallelism < newParallelism) {
-      addTasks(newParallelism);
-    } else if (oldParallelism > newParallelism) {
-      removeTasks(newParallelism);
-    }
-    Preconditions.checkState(this.numTasks == newParallelism, getLogIdentifier());
-    this.recoveredSourceEdgeProperties = sourceEdgeProperties;
-    this.recoveredRootInputSpecUpdates = rootInputSpecUpdates;
-  }
-
-  @Override
-  public VertexState restoreFromEvent(HistoryEvent historyEvent) {
-    writeLock.lock();
-    try {
-      switch (historyEvent.getEventType()) {
-        case VERTEX_INITIALIZED:
-          recoveryInitEventSeen = true;
-          recoveredState = setupVertex((VertexInitializedEvent) historyEvent);
-          createTasks();
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Recovered state for vertex after Init event"
-                + ", vertex=" + logIdentifier
-                + ", recoveredState=" + recoveredState);
-          }
-          return recoveredState;
-        case VERTEX_STARTED:
-          if (!recoveryInitEventSeen) {
-            throw new RuntimeException("Started Event seen but"
-                + " no Init Event was encountered earlier");
-          }
-          recoveryStartEventSeen = true;
-          VertexStartedEvent startedEvent = (VertexStartedEvent) historyEvent;
-          startTimeRequested = startedEvent.getStartRequestedTime();
-          startedTime = startedEvent.getStartTime();
-          recoveredState = VertexState.RUNNING;
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Recovered state for vertex after Started event"
-                + ", vertex=" + logIdentifier
-                + ", recoveredState=" + recoveredState);
-          }
-          return recoveredState;
-        case VERTEX_PARALLELISM_UPDATED:
-          // TODO TEZ-1019 this should flow through setParallelism method
-          VertexParallelismUpdatedEvent updatedEvent =
-              (VertexParallelismUpdatedEvent) historyEvent;
-          int oldNumTasks = numTasks;
-          int newNumTasks = updatedEvent.getNumTasks();
-          handleParallelismUpdate(newNumTasks, updatedEvent.getSourceEdgeProperties(),
-            updatedEvent.getRootInputSpecUpdates(), oldNumTasks);
-          Preconditions.checkState(this.numTasks == newNumTasks, getLogIdentifier());
-          if (updatedEvent.getVertexLocationHint() != null) {
-            setVertexLocationHint(updatedEvent.getVertexLocationHint());
-          }
-          stateChangeNotifier.stateChanged(vertexId,
-              new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Recovered state for vertex after parallelism updated event"
-                + ", vertex=" + logIdentifier
-                + ", recoveredState=" + recoveredState);
-          }
-          return recoveredState;
-        case VERTEX_COMMIT_STARTED:
-          recoveryCommitInProgress = true;
-          hasCommitter = true;
-          return recoveredState;
-        case VERTEX_FINISHED:
-          VertexFinishedEvent finishedEvent = (VertexFinishedEvent) historyEvent;
-          if (finishedEvent.isFromSummary()) {
-            summaryCompleteSeen  = true;
-          } else {
-            vertexCompleteSeen = true;
-          }
-          numTasks = finishedEvent.getNumTasks();
-          recoveryCommitInProgress = false;
-          recoveredState = finishedEvent.getState();
-          diagnostics.add(finishedEvent.getDiagnostics());
-          finishTime = finishedEvent.getFinishTime();
-          // TODO counters ??
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Recovered state for vertex after finished event"
-                + ", vertex=" + logIdentifier
-                + ", recoveredState=" + recoveredState);
-          }
-          return recoveredState;
-        case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-          VertexRecoverableEventsGeneratedEvent vEvent =
-              (VertexRecoverableEventsGeneratedEvent) historyEvent;
-          this.recoveredEvents.addAll(vEvent.getTezEvents());
-          return recoveredState;
-        default:
-          throw new RuntimeException("Unexpected event received for restoring"
-              + " state, eventType=" + historyEvent.getEventType());
-      }
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
   @Override
   public String getLogIdentifier() {
     return this.logIdentifier;
@@ -1613,7 +1455,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         if (!pendingTaskEvents.isEmpty()) {
           LOG.info("Routing pending task events for vertex: " + logIdentifier);
           try {
-            handleRoutedTezEvents(pendingTaskEvents, false, true);
+            handleRoutedTezEvents(pendingTaskEvents, true);
           } catch (AMUserCodeException e) {
             String msg = "Exception in " + e.getSource() + ", vertex=" + logIdentifier;
             LOG.error(msg, e);
@@ -1667,8 +1509,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         for (ScheduleTaskRequest task : tasksToSchedule) {
           TezTaskID taskId = TezTaskID.getInstance(vertexId, task.getTaskIndex());
           TaskSpec baseTaskSpec = createRemoteTaskSpec(taskId.getId());
+          boolean fromRecovery = recoveryData == null ? false : recoveryData.getTaskRecoveryData(taskId) != null;
           eventHandler.handle(new TaskEventScheduleTask(taskId, baseTaskSpec,
-              getTaskLocationHint(taskId)));
+              getTaskLocationHint(taskId), fromRecovery));
         }
       } finally {
         readLock.unlock();
@@ -1687,22 +1530,22 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
   public void reconfigureVertex(int parallelism,
       @Nullable VertexLocationHint locationHint,
       @Nullable Map<String, EdgeProperty> sourceEdgeProperties) throws AMUserCodeException {
-    setParallelism(parallelism, locationHint, sourceEdgeProperties, null, false, true);
+    setParallelismWrapper(parallelism, locationHint, sourceEdgeProperties, null, true);
   }
   
   @Override
   public void reconfigureVertex(@Nullable Map<String, InputSpecUpdate> rootInputSpecUpdate,
       int parallelism,
       @Nullable VertexLocationHint locationHint) throws AMUserCodeException {
-    setParallelism(parallelism, locationHint, null, rootInputSpecUpdate, false, true);
+    setParallelism(parallelism, locationHint, null, rootInputSpecUpdate, true);
   }
-  
+
   @Override
   public void reconfigureVertex(int parallelism,
       @Nullable VertexLocationHint locationHint,
       @Nullable Map<String, EdgeProperty> sourceEdgeProperties,
       @Nullable Map<String, InputSpecUpdate> rootInputSpecUpdate) throws AMUserCodeException {
-    setParallelism(parallelism, locationHint, sourceEdgeProperties, rootInputSpecUpdate, false, true);
+    setParallelismWrapper(parallelism, locationHint, sourceEdgeProperties, rootInputSpecUpdate, true);
   }
   
   @Override
@@ -1730,49 +1573,18 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     } finally {
       readLock.unlock();
     }
-    setParallelism(parallelism, vertexLocationHint, sourceEdgeProperties, rootInputSpecUpdates,
-        false, fromVertexManager);
+    setParallelismWrapper(parallelism, vertexLocationHint, sourceEdgeProperties, rootInputSpecUpdates,
+        fromVertexManager);
   }
 
-  private void setParallelism(int parallelism, VertexLocationHint vertexLocationHint,
+  private void setParallelismWrapper(int parallelism, VertexLocationHint vertexLocationHint,
       Map<String, EdgeProperty> sourceEdgeProperties,
       Map<String, InputSpecUpdate> rootInputSpecUpdates,
-      boolean recovering, boolean fromVertexManager) throws AMUserCodeException {
-    if (recovering) {
-      writeLock.lock();
-      try {
-        if (sourceEdgeProperties != null) {
-          for(Map.Entry<String, EdgeProperty> entry :
-            sourceEdgeProperties.entrySet()) {
-            LOG.info("Recovering edge manager for source:"
-                + entry.getKey() + " destination: " + getLogIdentifier());
-            Vertex sourceVertex = appContext.getCurrentDAG().getVertex(entry.getKey());
-            Edge edge = sourceVertices.get(sourceVertex);
-            try {
-              edge.setEdgeProperty(entry.getValue());
-            } catch (Exception e) {
-              throw new TezUncheckedException("Fail to setCustomEdgeManage for Edge,"
-                  + "sourceVertex:" + edge.getSourceVertexName()
-                  + "destinationVertex:" + edge.getDestinationVertexName(), e);
-            }
-          }
-        }
-
-        // Restore any rootInputSpecUpdates which may have been registered during a parallelism
-        // update.
-        if (rootInputSpecUpdates != null) {
-          LOG.info("Got updated RootInputsSpecs during recovery: " + rootInputSpecUpdates.toString());
-          this.rootInputSpecs.putAll(rootInputSpecUpdates);
-        }
-        return;
-      } finally {
-        writeLock.unlock();
-      }
-    }
+      boolean fromVertexManager) throws AMUserCodeException {
     Preconditions.checkArgument(parallelism >= 0, "Parallelism must be >=0. Value: " + parallelism
         + " for vertex: " + logIdentifier);
     writeLock.lock();
-
+    this.setParallelismCalledFlag = true;
     try {
       // disallow changing things after a vertex has started
       if (!tasksNotYetScheduled) {
@@ -1791,7 +1603,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
                 vertexToBeReconfiguredByManager,
                 "Vertex is fully configured but still"
                     + " the reconfiguration API has been called. VertexManager must notify the framework using "
-                    + " context.vertexReconfigurationPlanned() before re-configuring the vertex.");
+                    + " context.vertexReconfigurationPlanned() before re-configuring the vertex."
+                    + " vertexId=" + logIdentifier);
       }
       
       // Input initializer/Vertex Manager/1-1 split expected to set parallelism.
@@ -1875,7 +1688,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
             removeTasks(parallelism);
           }
         }
-
         Preconditions.checkState(this.numTasks == parallelism, getLogIdentifier());
         
         // set new vertex location hints
@@ -1903,13 +1715,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
           }
         }
 
-        // update history
-        VertexParallelismUpdatedEvent parallelismUpdatedEvent = new VertexParallelismUpdatedEvent(
-            vertexId, numTasks, vertexLocationHint, sourceEdgeProperties, rootInputSpecUpdates,
-            oldNumTasks);
-        appContext.getHistoryHandler().handle(
-            new DAGHistoryEvent(getDAGId(), parallelismUpdatedEvent));
-
         // stop buffering events
         for (Edge edge : sourceVertices.values()) {
           edge.stopEventBuffering();
@@ -1961,6 +1766,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         Preconditions.checkState(getInternalState() == VertexState.INITIALIZING, "Vertex: "
             + getLogIdentifier());
       }
+      
     } finally {
       writeLock.unlock();
     }    
@@ -2034,28 +1840,52 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
 
 
   void logJobHistoryVertexInitializedEvent() {
+    // TODO Vertex init may happen multiple times, so it is possible to have multiple VertexInitializedEvent
     VertexInitializedEvent initEvt = new VertexInitializedEvent(vertexId, vertexName,
         initTimeRequested, initedTime, numTasks,
-        getProcessorName(), getAdditionalInputs());
+        getProcessorName(), getAdditionalInputs(), initGeneratedEvents);
     this.appContext.getHistoryHandler().handle(
         new DAGHistoryEvent(getDAGId(), initEvt));
   }
 
   void logJobHistoryVertexStartedEvent() {
-    VertexStartedEvent startEvt = new VertexStartedEvent(vertexId,
-        startTimeRequested, startedTime);
-    this.appContext.getHistoryHandler().handle(
-        new DAGHistoryEvent(getDAGId(), startEvt));
+    if (recoveryData == null
+        || !recoveryData.isVertexStarted()) {
+      VertexStartedEvent startEvt = new VertexStartedEvent(vertexId,
+          startTimeRequested, startedTime);
+      this.appContext.getHistoryHandler().handle(
+          new DAGHistoryEvent(getDAGId(), startEvt));
+    }
+  }
+
+  void logVertexConfigurationDoneEvent() {
+    if (recoveryData == null || !recoveryData.shouldSkipInit()) {
+      Map<String, EdgeProperty> sourceEdgeProperties = new HashMap<String, EdgeProperty>();
+      for (Map.Entry<Vertex, Edge> entry : this.sourceVertices.entrySet()) {
+        sourceEdgeProperties.put(entry.getKey().getName(), entry.getValue().getEdgeProperty());
+      }
+      VertexConfigurationDoneEvent reconfigureDoneEvent =
+          new VertexConfigurationDoneEvent(vertexId, clock.getTime(),
+              numTasks, taskLocationHints == null ? null : VertexLocationHint.create(Lists.newArrayList(taskLocationHints)),
+                  sourceEdgeProperties, rootInputSpecs, setParallelismCalledFlag);
+      this.appContext.getHistoryHandler().handle(
+          new DAGHistoryEvent(getDAGId(), reconfigureDoneEvent));
+    }
   }
 
   void logJobHistoryVertexFinishedEvent() throws IOException {
-    this.setFinishTime();
-    logJobHistoryVertexCompletedHelper(VertexState.SUCCEEDED, finishTime, "");
+    if (recoveryData == null
+        || !recoveryData.isVertexSucceeded()) {
+      logJobHistoryVertexCompletedHelper(VertexState.SUCCEEDED, finishTime, "");
+    }
   }
 
   void logJobHistoryVertexFailedEvent(VertexState state) throws IOException {
-    logJobHistoryVertexCompletedHelper(state, clock.getTime(),
-        StringUtils.join(getDiagnostics(), LINE_SEPARATOR));
+    if (recoveryData == null
+        || !recoveryData.isVertexFinished()) {
+      logJobHistoryVertexCompletedHelper(state, clock.getTime(),
+          StringUtils.join(getDiagnostics(), LINE_SEPARATOR));
+    }
   }
 
   private void logJobHistoryVertexCompletedHelper(VertexState finalState, long finishTime,
@@ -2079,6 +1909,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     // commit only once. Dont commit shared outputs
     if (vertex.outputCommitters != null
         && !vertex.outputCommitters.isEmpty()) {
+      if (vertex.recoveryData != null
+          && vertex.recoveryData.isVertexCommitted()) {
+        LOG.info("Vertex was already committed as per recovery"
+            + " data, vertex=" + vertex.logIdentifier);
+        return vertex.finished(VertexState.SUCCEEDED);
+      }
       boolean firstCommit = true;
       for (Entry<String, OutputCommitter> entry : vertex.outputCommitters.entrySet()) {
         final OutputCommitter committer = entry.getValue();
@@ -2354,19 +2190,31 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
   }
 
   private boolean initializeVertex() {
-    try {
-      initializeCommitters();
-    } catch (Exception e) {
-      LOG.warn("Vertex Committer init failed, vertex=" + logIdentifier, e);
-      addDiagnostic("Vertex init failed : "
-          + ExceptionUtils.getStackTrace(e));
-      trySetTerminationCause(VertexTerminationCause.INIT_FAILURE);
-      finished(VertexState.FAILED);
-      return false;
+    // Don't need to initialize committer if vertex is fully completed
+    if (recoveryData != null && recoveryData.shouldSkipInit()) {
+      // Do other necessary recovery here
+      initedTime = recoveryData.getVertexInitedEvent().getInitedTime();
+      List<TezEvent> initGeneratedEvents = recoveryData.getVertexInitedEvent().getInitGeneratedEvents();
+      if (initGeneratedEvents != null && !initGeneratedEvents.isEmpty()) {
+        eventHandler.handle(new VertexEventRouteEvent(getVertexId(), initGeneratedEvents));
+      }
+    } else {
+      initedTime = clock.getTime();
+    }
+    // Only initialize committer when it is in non-recovery mode or vertex is not recovered to completed 
+    // state in recovery mode
+    if (recoveryData == null || recoveryData.getVertexFinishedEvent() == null) {
+      try {
+        initializeCommitters();
+      } catch (Exception e) {
+        LOG.warn("Vertex Committer init failed, vertex=" + logIdentifier, e);
+        addDiagnostic("Vertex init failed : "
+            + ExceptionUtils.getStackTrace(e));
+        trySetTerminationCause(VertexTerminationCause.INIT_FAILURE);
+        finished(VertexState.FAILED);
+        return false;
+      }
     }
-
-    // TODO: Metrics
-    initedTime = clock.getTime();
 
     logJobHistoryVertexInitializedEvent();
     return true;
@@ -2470,23 +2318,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     }
   }
 
-  private VertexState setupVertex() {
-    return setupVertex(null);
-  }
 
-  private VertexState setupVertex(VertexInitializedEvent event) {
+  private VertexState setupVertex() {
 
-    if (event == null) {
-      initTimeRequested = clock.getTime();
-    } else {
-      initTimeRequested = event.getInitRequestedTime();
-      initedTime = event.getInitedTime();
-    }
+    this.initTimeRequested = clock.getTime();
 
     // VertexManager needs to be setup before attempting to Initialize any
     // Inputs - since events generated by them will be routed to the
     // VertexManager for handling.
-
     if (dagVertexGroups != null && !dagVertexGroups.isEmpty()) {
       List<GroupInputSpec> groupSpecList = Lists.newLinkedList();
       for (VertexGroupInfo groupInfo : dagVertexGroups.values()) {
@@ -2502,24 +2341,20 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     }
 
     // Check if any inputs need initializers
-    if (event != null) {
-      this.rootInputDescriptors = event.getAdditionalInputs();
-    } else {
-      if (rootInputDescriptors != null) {
-        LOG.info("Root Inputs exist for Vertex: " + getName() + " : "
-            + rootInputDescriptors);
-        for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input
-            : rootInputDescriptors.values()) {
-          if (input.getControllerDescriptor() != null &&
-              input.getControllerDescriptor().getClassName() != null) {
-            if (inputsWithInitializers == null) {
-              inputsWithInitializers = Sets.newHashSet();
-            }
-            inputsWithInitializers.add(input.getName());
-            LOG.info("Starting root input initializer for input: "
-                + input.getName() + ", with class: ["
-                + input.getControllerDescriptor().getClassName() + "]");
+    if (rootInputDescriptors != null) {
+      LOG.info("Root Inputs exist for Vertex: " + getName() + " : "
+          + rootInputDescriptors);
+      for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input
+          : rootInputDescriptors.values()) {
+        if (input.getControllerDescriptor() != null &&
+            input.getControllerDescriptor().getClassName() != null) {
+          if (inputsWithInitializers == null) {
+            inputsWithInitializers = Sets.newHashSet();
           }
+          inputsWithInitializers.add(input.getName());
+          LOG.info("Starting root input initializer for input: "
+              + input.getName() + ", with class: ["
+              + input.getControllerDescriptor().getClassName() + "]");
         }
       }
     }
@@ -2536,13 +2371,21 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
 
     if (hasBipartite && inputsWithInitializers != null) {
       LOG.error("A vertex with an Initial Input and a Shuffle Input are not supported at the moment");
-      if (event != null) {
-        return VertexState.FAILED;
-      } else {
-        return finished(VertexState.FAILED);
-      }
+      return finished(VertexState.FAILED);
+    }
+
+    numTasks = getVertexPlan().getTaskConfig().getNumTasks();
+    if (!(numTasks == -1 || numTasks >= 0)) {
+      addDiagnostic("Invalid task count for vertex"
+          + ", numTasks=" + numTasks);
+      trySetTerminationCause(VertexTerminationCause.INVALID_NUM_OF_TASKS);
+      return VertexState.FAILED;
     }
 
+    checkTaskLimits();
+    // set VertexManager as the last step. Because in recovery case, we may need to restore 
+    // some info from last the AM attempt and skip the initialization step. Otherwise numTasks may be
+    // reset to -1 after the restore.
     try {
       assignVertexManager();
     } catch (TezException e1) {
@@ -2571,38 +2414,43 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
           msg + ", " + e.getMessage() + ", " + ExceptionUtils.getStackTrace(e.getCause()));
       return VertexState.FAILED;
     }
-
-    // Setup tasks early if possible. If the VertexManager is not being used
-    // to set parallelism, sending events to Tasks is safe (and less confusing
-    // then relying on tasks to be created after TaskEvents are generated).
-    // For VertexManagers setting parallelism, the setParallelism call needs
-    // to be inline.
-    if (event != null) {
-      int oldNumTasks = numTasks;
-      numTasks = event.getNumTasks();
-      stateChangeNotifier.stateChanged(vertexId,
-          new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
-    } else {
-      numTasks = getVertexPlan().getTaskConfig().getNumTasks();
-      // Not sending a parallelism update notification since this is from the original plan
-    }
-
-    if (!(numTasks == -1 || numTasks >= 0)) {
-      addDiagnostic("Invalid task count for vertex"
-          + ", numTasks=" + numTasks);
-      trySetTerminationCause(VertexTerminationCause.INVALID_NUM_OF_TASKS);
-      if (event != null) {
-        return finished(VertexState.FAILED);
-      } else {
-        return VertexState.FAILED;
-      }
-    }
-    
-    checkTaskLimits();
     return VertexState.INITED;
   }
 
   private void assignVertexManager() throws TezException {
+    // condition for skip initializing stage
+    //   - VertexInputInitializerEvent is seen
+    //   - VertexReconfigureDoneEvent is seen
+    //        -  Reason to check whether VertexManager has complete its responsibility
+    //           VertexManager actually is involved in the InputInitializer (InputInitializer generate events
+    //           and send them to VertexManager which do some processing and send back to Vertex), so that means
+    //           Input initializer will affect on the VertexManager and we couldn't skip the initializing step if  
+    //           VertexManager has not completed its responsibility.
+    //        -  Why using VertexReconfigureDoneEvent
+    //           -  VertexReconfigureDoneEvent represent the case that user use API reconfigureVertex
+    //              VertexReconfigureDoneEvent will be logged
+    if (recoveryData != null
+        && recoveryData.shouldSkipInit()) {
+      // Replace the original VertexManager with NoOpVertexManager if the reconfiguration is done in the last AM attempt
+      VertexConfigurationDoneEvent reconfigureDoneEvent = recoveryData.getVertexConfigurationDoneEvent();
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("VertexManager reconfiguration is done in the last AM Attempt"
+            + ", use NoOpVertexManager to replace it, vertexId=" + logIdentifier);
+        LOG.debug("VertexReconfigureDoneEvent=" + reconfigureDoneEvent);
+      }
+      ByteArrayOutputStream out = new ByteArrayOutputStream();
+      try {
+        reconfigureDoneEvent.toProtoStream(out);
+      } catch (IOException e) {
+        throw new TezUncheckedException("Unable to deserilize VertexReconfigureDoneEvent");
+      }
+      this.vertexManager = new VertexManager(
+          VertexManagerPluginDescriptor.create(NoOpVertexManager.class.getName())
+            .setUserPayload(UserPayload.create(ByteBuffer.wrap(out.toByteArray()))),
+          dagUgi, this, appContext, stateChangeNotifier);
+      return;
+    }
+
     boolean hasBipartite = false;
     boolean hasOneToOne = false;
     boolean hasCustom = false;
@@ -2671,258 +2519,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       }
     }
   }
-
-  public static class StartRecoverTransition implements
-      MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-
-    @Override
-    public VertexState transition(VertexImpl vertex, VertexEvent vertexEvent) {
-      VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vertexEvent;
-      VertexState desiredState = recoverEvent.getDesiredState();
-
-      switch (desiredState) {
-        case RUNNING:
-          break;
-        case SUCCEEDED:
-        case KILLED:
-        case FAILED:
-        case ERROR:
-          if (desiredState == VertexState.SUCCEEDED) {
-            vertex.succeededTaskCount = vertex.numTasks;
-            vertex.completedTaskCount = vertex.numTasks;
-          } else if (desiredState == VertexState.KILLED) {
-            vertex.killedTaskCount = vertex.numTasks;
-          } else if (desiredState == VertexState.FAILED || desiredState == VertexState.ERROR) {
-            vertex.failedTaskCount = vertex.numTasks;
-          }
-          if (vertex.tasks != null) {
-            TaskState taskState = TaskState.KILLED;
-            if (desiredState == VertexState.SUCCEEDED) {
-              taskState = TaskState.SUCCEEDED;
-            } else if (desiredState == VertexState.KILLED) {
-              taskState = TaskState.KILLED;
-            } else if (desiredState == VertexState.FAILED || desiredState == VertexState.ERROR) {
-              taskState = TaskState.FAILED;
-            }
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId(),
-                      taskState, false));
-            }
-          }
-          LOG.info("DAG informed Vertex of its final completed state"
-              + ", vertex=" + vertex.logIdentifier
-              + ", state=" + desiredState);
-          return desiredState;
-        default:
-          LOG.info("Unhandled desired state provided by DAG"
-              + ", vertex=" + vertex.logIdentifier
-              + ", state=" + desiredState);
-          vertex.finished(VertexState.ERROR);
-      }
-
-      // recover from recover log, should recover to running
-      // desiredState must be RUNNING based on above code
-      VertexState endState;
-      switch (vertex.recoveredState) {
-        case NEW:
-          // Trigger init and start as desired state is RUNNING
-          // Drop all root events
-          Iterator<TezEvent> iterator = vertex.recoveredEvents.iterator();
-          while (iterator.hasNext()) {
-            if (iterator.next().getEventType().equals(
-                EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
-              iterator.remove();
-            }
-          }
-          vertex.eventHandler.handle(new VertexEvent(vertex.vertexId,
-              VertexEventType.V_INIT));
-          vertex.eventHandler.handle(new VertexEvent(vertex.vertexId,
-              VertexEventType.V_START));
-          endState = VertexState.NEW;
-          break;
-        case INITED:
-          try {
-            vertex.initializeCommitters();
-          } catch (Exception e) {
-            String msg = "Failed to initialize committers"
-                + ", vertex=" + vertex.logIdentifier + ","
-                + ExceptionUtils.getStackTrace(e);
-            LOG.error(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.INIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-            break;
-          }
-
-          // Recover tasks
-          if (vertex.tasks != null) {
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId()));
-            }
-          }
-          // Update tasks with their input payloads as needed
-
-          vertex.eventHandler.handle(new VertexEvent(vertex.vertexId,
-              VertexEventType.V_START));
-          if (vertex.getInputVertices().isEmpty()) {
-            endState = VertexState.INITED;
-          } else {
-            endState = VertexState.RECOVERING;
-          }
-          break;
-        case RUNNING:
-          try {
-            vertex.initializeCommitters();
-          } catch (Exception e) {
-            String msg = "Failed to initialize committers"
-                + ", vertex=" + vertex.logIdentifier + ","
-                + ExceptionUtils.getStackTrace(e);
-            LOG.error(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.INIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-            break;
-          }
-
-          // if commit in progress and desired state is not a succeeded one,
-          // move to failed
-          if (vertex.recoveryCommitInProgress) {
-            String msg = "Recovered vertex was in the middle of a commit"
-                + ", failing Vertex=" + vertex.logIdentifier;
-            LOG.warn(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.COMMIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-            break;
-          }
-          assert vertex.tasks.size() == vertex.numTasks;
-          if (vertex.tasks != null && vertex.numTasks != 0) {
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId()));
-            }
-            try {
-              vertex.recoveryCodeSimulatingStart();
-              vertex.unsetTasksNotYetScheduled();
-              endState = VertexState.RUNNING;
-            } catch (AMUserCodeException e) {
-              String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier();
-              LOG.error(msg, e);
-              vertex.finished(VertexState.FAILED, VertexTerminationCause.AM_USERCODE_FAILURE,
-                  msg + ", " + ExceptionUtils.getStackTrace(e.getCause()));
-              endState = VertexState.FAILED;
-            }
-          } else {
-            // why succeeded here
-            endState = VertexState.SUCCEEDED;
-            vertex.finished(endState);
-          }
-          break;
-        case SUCCEEDED:
-          if (vertex.hasCommitter
-              && vertex.summaryCompleteSeen && !vertex.vertexCompleteSeen) {
-            String msg = "Cannot recover vertex as all recovery events not"
-                + " found, vertex=" + vertex.logIdentifier
-                + ", hasCommitters=" + vertex.hasCommitter
-                + ", summaryCompletionSeen=" + vertex.summaryCompleteSeen
-                + ", finalCompletionSeen=" + vertex.vertexCompleteSeen;
-            LOG.warn(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.COMMIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-          } else {
-            // recover tasks
-            if (vertex.tasks != null && vertex.numTasks != 0) {
-              TaskState taskState = TaskState.SUCCEEDED;
-              for (Task task : vertex.tasks.values()) {
-                vertex.eventHandler.handle(
-                    new TaskEventRecoverTask(task.getTaskId(),
-                        taskState));
-              }
-              try {
-                vertex.recoveryCodeSimulatingStart();
-                vertex.unsetTasksNotYetScheduled();
-                endState = VertexState.RUNNING;
-              } catch (AMUserCodeException e) {
-                String msg = "Exception in " + e.getSource() +", vertex:" + vertex.getLogIdentifier();
-                LOG.error(msg, e);
-                vertex.finished(VertexState.FAILED, VertexTerminationCause.AM_USERCODE_FAILURE,
-                    msg + "," + ExceptionUtils.getStackTrace(e.getCause()));
-                endState = VertexState.FAILED;
-              }
-            } else {
-              endState = vertex.recoveredState;
-              vertex.finished(endState);
-            }
-          }
-          break;
-        case FAILED:
-        case KILLED:
-          // vertex may be killed/failed before its tasks are scheduled. so here just recover vertex
-          // to the recovered state without waiting for its tasks' feedback and recover tasks to
-          // the corresponding state without recover its data.
-          if (vertex.tasks != null && vertex.numTasks != 0) {
-            TaskState taskState = TaskState.FAILED;
-            if (vertex.recoveredState == VertexState.KILLED) {
-              taskState = TaskState.KILLED;
-            }
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId(),
-                      taskState, false));
-            }
-          }
-          endState = vertex.recoveredState;
-          vertex.finished(endState);
-          break;
-        default:
-          LOG.warn("Invalid recoveredState found when trying to recover"
-              + " vertex"
-              + ", vertex=" + vertex.logIdentifier
-              + ", recoveredState=" + vertex.recoveredState);
-          vertex.finished(VertexState.ERROR);
-          endState = VertexState.ERROR;
-          break;
-      }
-      if (!endState.equals(VertexState.RECOVERING)) {
-        LOG.info("Recovered Vertex State"
-            + ", vertexId=" + vertex.logIdentifier
-            + ", state=" + endState
-            + ", numInitedSourceVertices=" + vertex.numInitedSourceVertices
-            + ", numStartedSourceVertices=" + vertex.numStartedSourceVertices
-            + ", numRecoveredSourceVertices=" + vertex.numRecoveredSourceVertices
-            + ", recoveredEvents="
-            + ( vertex.recoveredEvents == null ? "null" : vertex.recoveredEvents.size())
-            + ", tasksIsNull=" + (vertex.tasks == null)
-            + ", numTasks=" + ( vertex.tasks == null ? "null" : vertex.tasks.size()));
-        for (Entry<Vertex, Edge> entry : vertex.getOutputVertices().entrySet()) {
-          vertex.eventHandler.handle(new VertexEventSourceVertexRecovered(
-              entry.getKey().getVertexId(),
-              vertex.vertexId, endState, null,
-              vertex.getDistanceFromRoot()));
-        }
-      }
-      if (EnumSet.of(VertexState.RUNNING, VertexState.SUCCEEDED, VertexState.INITED)
-          .contains(endState)) {
-        // Send events downstream
-        vertex.routeRecoveredEvents(endState, vertex.recoveredEvents);
-        vertex.recoveredEvents.clear();
-      } else {
-        // Ensure no recovered events
-        if (!vertex.recoveredEvents.isEmpty()) {
-          throw new RuntimeException("Invalid Vertex state"
-              + ", found non-zero recovered events in invalid state"
-              + ", vertex=" + vertex.logIdentifier
-              + ", recoveredState=" + endState
-              + ", recoveredEvents=" + vertex.recoveredEvents.size());
-        }
-      }
-      return endState;
-    }
-
-  }
   
   private static List<TaskAttemptIdentifier> getTaskAttemptIdentifiers(DAG dag, 
       List<TezTaskAttemptID> taIds) {
@@ -2939,73 +2535,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       TezTaskAttemptID taId) {
     return new TaskAttemptIdentifierImpl(dagName, vertexName, taId);
   }
-  
-  private void recoveryCodeSimulatingStart() throws AMUserCodeException {
-    vertexManager.onVertexStarted(getTaskAttemptIdentifiers(dag, pendingReportedSrcCompletions));
-    // This code is duplicated from startVertex() because recovery does not follow normal
-    // transitions. To be removed after recovery code is fixed.
-    maybeSendConfiguredEvent();
-  }
-
-  private void routeRecoveredEvents(VertexState vertexState,
-      List<TezEvent> tezEvents) {
-    for (TezEvent tezEvent : tezEvents) {
-      EventMetaData sourceMeta = tezEvent.getSourceInfo();
-      TezTaskAttemptID srcTaId = sourceMeta.getTaskAttemptID();
-      if (tezEvent.getEventType() == EventType.DATA_MOVEMENT_EVENT) {
-        ((DataMovementEvent) tezEvent.getEvent()).setVersion(srcTaId.getId());
-      } else if (tezEvent.getEventType() == EventType.COMPOSITE_DATA_MOVEMENT_EVENT) {
-        ((CompositeDataMovementEvent) tezEvent.getEvent()).setVersion(srcTaId.getId());
-      } else if (tezEvent.getEventType() == EventType.INPUT_FAILED_EVENT) {
-        ((InputFailedEvent) tezEvent.getEvent()).setVersion(srcTaId.getId());
-      } else if (tezEvent.getEventType() == EventType.ROOT_INPUT_DATA_INFORMATION_EVENT) {
-        if (vertexState == VertexState.RUNNING
-            || vertexState == VertexState.INITED) {
-          // Only routed if vertex is still running
-          eventHandler.handle(new VertexEventRouteEvent(
-              this.getVertexId(), Collections.singletonList(tezEvent), true));
-        }
-        continue;
-      } else if (tezEvent.getEventType() == EventType.ROOT_INPUT_INITIALIZER_EVENT) {
-        // The event has the relevant target information
-        InputInitializerEvent iiEvent = (InputInitializerEvent) tezEvent.getEvent();
-        iiEvent.setSourceVertexName(vertexName);
-        eventHandler.handle(new VertexEventRouteEvent(
-            getDAG().getVertex(iiEvent.getTargetVertexName()).getVertexId(),
-            Collections.singletonList(tezEvent), true));
-        continue;
-      }
-
-      Vertex destVertex = getDAG().getVertex(sourceMeta.getEdgeVertexName());
-      Edge destEdge = targetVertices.get(destVertex);
-      if (destEdge == null) {
-        throw new TezUncheckedException("Bad destination vertex: " +
-            sourceMeta.getEdgeVertexName() + " for event vertex: " +
-            getLogIdentifier());
-      }
-      if (LOG.isDebugEnabled()) {
-        LOG.debug("Routing recovered event"
-            + ", vertex=" + logIdentifier
-            + ", eventType=" + tezEvent.getEventType()
-            + ", sourceInfo=" + sourceMeta
-            + ", destinationVertex=" + destVertex.getLogIdentifier());
-      }
-      eventHandler.handle(new VertexEventRouteEvent(destVertex
-          .getVertexId(), Collections.singletonList(tezEvent), true));
-    }
-  }
-
-  public static class TerminateDuringRecoverTransition implements
-      SingleArcTransition<VertexImpl, VertexEvent> {
-
-    @Override
-    public void transition(VertexImpl vertex, VertexEvent vertexEvent) {
-      LOG.info("Received a terminate during recovering, setting recovered"
-          + " state to KILLED");
-      vertex.recoveredState = VertexState.KILLED;
-    }
-
-  }
 
   public static class NullEdgeInitializedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
@@ -3036,375 +2565,64 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     }
   }
 
-  public static class BufferDataRecoverTransition implements
-      SingleArcTransition<VertexImpl, VertexEvent> {
-
-    @Override
-    public void transition(VertexImpl vertex, VertexEvent vertexEvent) {
-      LOG.info("Received upstream event while still recovering"
-          + ", vertexId=" + vertex.logIdentifier
-          + ", vertexEventType=" + vertexEvent.getType());
-      if (vertexEvent.getType().equals(VertexEventType.V_ROUTE_EVENT)) {
-        VertexEventRouteEvent evt = (VertexEventRouteEvent) vertexEvent;
-        vertex.pendingRouteEvents.addAll(evt.getEvents());
-      } else if (vertexEvent.getType().equals(
-          VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED)) {
-        VertexEventSourceTaskAttemptCompleted evt =
-            (VertexEventSourceTaskAttemptCompleted) vertexEvent;
-        vertex.pendingReportedSrcCompletions.add(
-            evt.getCompletionEvent().getTaskAttemptId());
-      } else if (vertexEvent.getType().equals(
-          VertexEventType.V_SOURCE_VERTEX_STARTED)) {
-        VertexEventSourceVertexStarted startEvent =
-            (VertexEventSourceVertexStarted) vertexEvent;
-        int distanceFromRoot = startEvent.getSourceDistanceFromRoot() + 1;
-        if(vertex.distanceFromRoot < distanceFromRoot) {
-          vertex.distanceFromRoot = distanceFromRoot;
-        }
-        ++vertex.numStartedSourceVertices;
-      } else if (vertexEvent.getType().equals(VertexEventType.V_INIT)) {
-        ++vertex.numInitedSourceVertices;
-      }
-    }
-  }
-
-
   public static class RecoverTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
 
     @Override
     public VertexState transition(VertexImpl vertex, VertexEvent vertexEvent) {
-      VertexEventSourceVertexRecovered sourceRecoveredEvent =
-          (VertexEventSourceVertexRecovered) vertexEvent;
-      // Use distance from root from Recovery events as upstream vertices may not
-      // send source vertex started event that is used to compute distance
-      int distanceFromRoot = sourceRecoveredEvent.getSourceDistanceFromRoot() + 1;
-      if(vertex.distanceFromRoot < distanceFromRoot) {
-        vertex.distanceFromRoot = distanceFromRoot;
-      }
-
-      ++vertex.numRecoveredSourceVertices;
-
-      switch (sourceRecoveredEvent.getSourceVertexState()) {
-        case NEW:
-          // Nothing to do
-          break;
-        case INITED:
-          ++vertex.numInitedSourceVertices;
-          break;
-        case RUNNING:
-        case SUCCEEDED:
-          ++vertex.numInitedSourceVertices;
-          ++vertex.numStartedSourceVertices;
-          if (sourceRecoveredEvent.getCompletedTaskAttempts() != null) {
-            vertex.pendingReportedSrcCompletions.addAll(
-                sourceRecoveredEvent.getCompletedTaskAttempts());
-          }
-          break;
-        case FAILED:
-        case KILLED:
-        case ERROR:
-          // Nothing to do
-          // Recover as if source vertices have not inited/started
-          break;
-        default:
-          LOG.warn("Received invalid SourceVertexRecovered event"
-              + ", vertex=" + vertex.logIdentifier
-              + ", sourceVertex=" + sourceRecoveredEvent.getSourceVertexID()
-              + ", sourceVertexState=" + sourceRecoveredEvent.getSourceVertexState());
-          return vertex.finished(VertexState.ERROR);
-      }
-
-      if (vertex.numRecoveredSourceVertices !=
-          vertex.getInputVerticesCount()) {
-        if (LOG.isDebugEnabled()) {
-          LOG.debug("Waiting for source vertices to recover"
-              + ", vertex=" + vertex.logIdentifier
-              + ", numRecoveredSourceVertices=" + vertex.numRecoveredSourceVertices
-              + ", totalSourceVertices=" + vertex.getInputVerticesCount());
-        }
-        return VertexState.RECOVERING;
-      }
-
-
-      // Complete recovery
-      VertexState endState = VertexState.NEW;
-      List<TezTaskAttemptID> completedTaskAttempts = Lists.newLinkedList();
-      switch (vertex.recoveredState) {
-        case NEW:
-          // Drop all root events if not inited properly
-          Iterator<TezEvent> iterator = vertex.recoveredEvents.iterator();
-          while (iterator.hasNext()) {
-            if (iterator.next().getEventType().equals(
-                EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)) {
-              iterator.remove();
-            }
-          }
-          // Trigger init if all sources initialized
-          if (vertex.numInitedSourceVertices == vertex.getInputVerticesCount()) {
-            vertex.eventHandler.handle(new VertexEvent(vertex.vertexId,
-                VertexEventType.V_INIT));
-          }
-          if (vertex.numStartedSourceVertices == vertex.getInputVerticesCount()) {
-            vertex.eventHandler.handle(new VertexEvent(vertex.vertexId,
-                VertexEventType.V_START));
-          }
-          endState = VertexState.NEW;
-          break;
-        case INITED:
-          vertex.vertexAlreadyInitialized = true;
-          try {
-            vertex.initializeCommitters();
-          } catch (Exception e) {
-            String msg = "Failed to initialize committers, vertex="
-                + vertex.logIdentifier + "," + ExceptionUtils.getStackTrace(e);
-            LOG.error(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.INIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-            break;
-          }
-          boolean successSetParallelism ;
-          try {
-            // recovering only edge manager
-            vertex.setParallelism(0,
-              null, vertex.recoveredSourceEdgeProperties, vertex.recoveredRootInputSpecUpdates, true, false);
-            successSetParallelism = true;
-          } catch (Exception e) {
-            successSetParallelism = false;
-          }
-          if (!successSetParallelism) {
-            String msg  = "Failed to recover edge managers, vertex=" + vertex.logIdentifier;
-            LOG.error(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.INIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-            break;
-          }
-          // Recover tasks
-          if (vertex.tasks != null) {
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId()));
-            }
-          }
-          if (vertex.numInitedSourceVertices != vertex.getInputVerticesCount()) {
-            LOG.info("Vertex already initialized but source vertices have not"
-                + " initialized"
-                + ", vertexId=" + vertex.logIdentifier
-                + ", numInitedSourceVertices=" + vertex.numInitedSourceVertices);
-          } else {
-            if (vertex.numStartedSourceVertices == vertex.getInputVerticesCount()) {
-              vertex.eventHandler.handle(new VertexEvent(vertex.vertexId,
-                VertexEventType.V_START));
-            }
-          }
-          endState = VertexState.INITED;
-          break;
-        case RUNNING:
-          // if commit in progress and desired state is not a succeeded one,
-          // move to failed
-          if (vertex.recoveryCommitInProgress) {
-            LOG.info("Recovered vertex was in the middle of a commit"
-                + ", failing Vertex=" + vertex.logIdentifier);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.COMMIT_FAILURE, null);
-            endState = VertexState.FAILED;
-            break;
-          }
-          try {
-            vertex.initializeCommitters();
-          } catch (Exception e) {
-            String msg = "Failed to initialize committers, vertex="
-                + vertex.logIdentifier + "," + ExceptionUtils.getStackTrace(e);
-            LOG.error(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.INIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-            break;
-          }
-          try {
-            vertex.setParallelism(vertex.numTasks, null, vertex.recoveredSourceEdgeProperties,
-              vertex.recoveredRootInputSpecUpdates, true, false);
-            successSetParallelism = true;
-          } catch (Exception e) {
-            successSetParallelism = false;
-          }
-          if (!successSetParallelism) {
-            String msg = "Failed to recover edge managers for vertex:" + vertex.logIdentifier;
-            LOG.error(msg);
-            vertex.finished(VertexState.FAILED,
-                VertexTerminationCause.INIT_FAILURE, msg);
-            endState = VertexState.FAILED;
-            break;
-          }
-          assert vertex.tasks.size() == vertex.numTasks;
-          if (vertex.tasks != null && vertex.numTasks != 0) {
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId()));
-            }
-            try {
-              vertex.recoveryCodeSimulatingStart();
-              vertex.unsetTasksNotYetScheduled();
-              endState = VertexState.RUNNING;
-            } catch (AMUserCodeException e) {
-              String msg = "Exception in " + e.getSource() + ", vertex=" + vertex.getLogIdentifier();
-              LOG.error(msg, e);
-              vertex.finished(VertexState.FAILED, VertexTerminationCause.AM_USERCODE_FAILURE,
-                  msg + "," + ExceptionUtils.getStackTrace(e.getCause()));
-              endState = VertexState.FAILED;
-            }
-          } else {
-            endState = VertexState.SUCCEEDED;
-            vertex.finished(endState);
-          }
-          break;
-        case SUCCEEDED:
-          // recover tasks
-          assert vertex.tasks.size() == vertex.numTasks;
-          if (vertex.tasks != null  && vertex.numTasks != 0) {
-            TaskState taskState = TaskState.SUCCEEDED;
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId(),
-                      taskState));
-            }
-            // Wait for all tasks to recover and report back
-            try {
-              vertex.recoveryCodeSimulatingStart();
-              vertex.unsetTasksNotYetScheduled();
-              endState = VertexState.RUNNING;
-            } catch (AMUserCodeException e) {
-              String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier();
-              LOG.error(msg, e);
-              vertex.finished(VertexState.FAILED, VertexTerminationCause.AM_USERCODE_FAILURE,
-                  msg + "," + ExceptionUtils.getStackTrace(e.getCause()));
-              endState = VertexState.FAILED;
-            }
-          } else {
-            endState = vertex.recoveredState;
-            vertex.finished(endState);
-          }
-          break;
-        case FAILED:
-        case KILLED:
-          // vertex may be killed/failed before its tasks are scheduled. so here just recover vertex
-          // to the recovered state without waiting for its tasks' feedback and recover tasks to
-          // the corresponding state without recover its data.
-          if (vertex.tasks != null && vertex.numTasks != 0) {
-            TaskState taskState = TaskState.FAILED;
-            if (vertex.recoveredState == VertexState.KILLED) {
-              taskState = TaskState.KILLED;
-            }
-            for (Task task : vertex.tasks.values()) {
-              vertex.eventHandler.handle(
-                  new TaskEventRecoverTask(task.getTaskId(),
-                      taskState, false));
-            }
-          }
-          endState = vertex.recoveredState;
-          vertex.finished(endState);
-          break;
-        default:
-          LOG.warn("Invalid recoveredState found when trying to recover"
-              + " vertex, recoveredState=" + vertex.recoveredState);
-          vertex.finished(VertexState.ERROR);
-          endState = VertexState.ERROR;
-          break;
-      }
-
-      LOG.info("Recovered Vertex State"
-          + ", vertexId=" + vertex.logIdentifier
-          + ", state=" + endState
-          + ", numInitedSourceVertices" + vertex.numInitedSourceVertices
-          + ", numStartedSourceVertices=" + vertex.numStartedSourceVertices
-          + ", numRecoveredSourceVertices=" + vertex.numRecoveredSourceVertices
-          + ", tasksIsNull=" + (vertex.tasks == null)
-          + ", numTasks=" + ( vertex.tasks == null ? 0 : vertex.tasks.size()));
-
-      for (Entry<Vertex, Edge> entry : vertex.getOutputVertices().entrySet()) {
-        vertex.eventHandler.handle(new VertexEventSourceVertexRecovered(
-            entry.getKey().getVertexId(),
-            vertex.vertexId, endState, completedTaskAttempts,
-            vertex.getDistanceFromRoot()));
-      }
-      if (EnumSet.of(VertexState.RUNNING, VertexState.SUCCEEDED, VertexState.INITED)
-          .contains(endState)) {
-        // Send events downstream
-        vertex.routeRecoveredEvents(endState, vertex.recoveredEvents);
-        vertex.recoveredEvents.clear();
-        if (!vertex.pendingRouteEvents.isEmpty()) {
-          try {
-            vertex.handleRoutedTezEvents(vertex.pendingRouteEvents, false, true);
-            vertex.pendingRouteEvents.clear();
-          } catch (AMUserCodeException e) {
-            String msg = "Exception in " + e.getSource() + ", vertex=" + vertex.getLogIdentifier();
-            LOG.error(msg, e);
-            vertex.finished(VertexState.FAILED, VertexTerminationCause.AM_USERCODE_FAILURE,
-                msg + ", " + e.getMessage() + ", " + ExceptionUtils.getStackTrace(e.getCause()));
-            endState = VertexState.FAILED;
-          }
-        }
-      } else {
-        // Ensure no recovered events
-        if (!vertex.recoveredEvents.isEmpty()) {
-          throw new RuntimeException("Invalid Vertex state"
-              + ", found non-zero recovered events in invalid state"
-              + ", recoveredState=" + endState
-              + ", recoveredEvents=" + vertex.recoveredEvents.size());
-        }
+      VertexEventRecoverVertex recoverEvent = (VertexEventRecoverVertex) vertexEvent;
+      // with desired state, for the cases that DAG is completed
+      VertexState desiredState = recoverEvent.getDesiredState();
+      switch (desiredState) {
+      case SUCCEEDED:
+        vertex.succeededTaskCount = vertex.numTasks;
+        vertex.completedTaskCount = vertex.numTasks;
+        break;
+      case KILLED:
+        vertex.killedTaskCount = vertex.numTasks;
+        break;
+      case FAILED:
+      case ERROR:
+        vertex.failedTaskCount = vertex.numTasks;
+        break;
+      default:
+        LOG.info("Unhandled desired state provided by DAG"
+            + ", vertex=" + vertex.logIdentifier
+            + ", state=" + desiredState);
+        return vertex.finished(VertexState.ERROR);
       }
-      return endState;
-    }
-
-  }
 
-  public static class IgnoreInitInInitedTransition implements
-      MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-
-    @Override
-    public VertexState transition(VertexImpl vertex, VertexEvent event) {
-      LOG.info("Received event during INITED state"
+      LOG.info("DAG informed vertices of its final completed state"
           + ", vertex=" + vertex.logIdentifier
-          + ", eventType=" + event.getType());
-      if (!vertex.vertexAlreadyInitialized) {
-        LOG.error("Vertex not initialized but in INITED state"
-            + ", vertexId=" + vertex.logIdentifier);
-        return vertex.finished(VertexState.ERROR);
-      } else {
-        return VertexState.INITED;
-      }
+          + ", desiredState=" + desiredState);
+      return vertex.finished(recoverEvent.getDesiredState());
     }
   }
-
-
-
+  
   public static class InitTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
 
     @Override
     public VertexState transition(VertexImpl vertex, VertexEvent event) {
+      // recover from recovery data (NEW->FAILED/KILLED)
+      if (vertex.recoveryData != null
+          && !vertex.recoveryData.isVertexInited()
+          && vertex.recoveryData.isVertexFinished()) {
+        VertexFinishedEvent finishedEvent = vertex.recoveryData.getVertexFinishedEvent();
+        vertex.diagnostics.add(finishedEvent.getDiagnostics());
+        return vertex.finished(finishedEvent.getState());
+      }
+
       VertexState vertexState = VertexState.NEW;
       vertex.numInitedSourceVertices++;
-      // TODO fix this as part of TEZ-1008
-      // Should have a different way to infer source vertices INITED
-      // as compared to a recovery triggered INIT
-      // In normal flow, upstream vertices send a V_INIT downstream to
-      // trigger an init of the downstream vertex. In case of recovery,
-      // upstream vertices may not send this event if they are already in a
-      // RUNNING or completed state. Hence, recovering vertices may send
-      // themselves a V_INIT to trigger a transition. Hence, the count may
-      // go one over.
       if (vertex.sourceVertices == null || vertex.sourceVertices.isEmpty() ||
-          (vertex.numInitedSourceVertices == vertex.sourceVertices.size()
-            || vertex.numInitedSourceVertices == (vertex.sourceVertices.size()+1))) {
-        vertexState = handleInitEvent(vertex, event);
+          (vertex.numInitedSourceVertices == vertex.sourceVertices.size())) {
+        vertexState = handleInitEvent(vertex);
         if (vertexState != VertexState.FAILED) {
           if (vertex.targetVertices != null && !vertex.targetVertices.isEmpty()) {
             for (Vertex target : vertex.targetVertices.keySet()) {
               vertex.getEventHandler().handle(new VertexEvent(target.getVertexId(),
-                VertexEventType.V_INIT));
+                  VertexEventType.V_INIT));
             }
           }
         }
@@ -3412,11 +2630,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       return vertexState;
     }
 
-    private VertexState handleInitEvent(VertexImpl vertex, VertexEvent event) {
+    private VertexState handleInitEvent(VertexImpl vertex) {
       VertexState state = vertex.setupVertex();
       if (state.equals(VertexState.FAILED)) {
         return state;
       }
+
       // TODO move before to handle NEW state
       if (vertex.targetVertices != null) {
         for (Edge e : vertex.targetVertices.values()) {
@@ -3448,13 +2667,15 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
             + " to set #tasks for the vertex " + vertex.getLogIdentifier());
 
         if (vertex.inputsWithInitializers != null) {
-          LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier);
-          try {
-            vertex.setupInputInitializerManager();
-          } catch (TezException e) {
-            String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e);
-            LOG.info(msg);
-            return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg);
+          if (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit()) {
+            LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier);
+            try {
+              vertex.setupInputInitializerManager();
+            } catch (TezException e) {
+              String msg = "Fail to create InputInitializerManager, " + ExceptionUtils.getStackTrace(e);
+              LOG.info(msg);
+              return vertex.finished(VertexState.FAILED, VertexTerminationCause.INIT_FAILURE, msg);
+            }
           }
           return VertexState.INITIALIZING;
         } else {
@@ -3484,7 +2705,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
         LOG.info("Creating " + vertex.numTasks + " tasks for vertex: " + vertex.logIdentifier);
         vertex.createTasks();
         // this block may return VertexState.INITIALIZING
-        if (vertex.inputsWithInitializers != null) {
+        if (vertex.inputsWithInitializers != null &&
+            (vertex.recoveryData == null || !vertex.recoveryData.shouldSkipInit())) {
           LOG.info("Vertex will initialize from input initializer. " + vertex.logIdentifier);
           try {
             vertex.setupInputInitializerManager();
@@ -3608,7 +2830,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       List<TezEvent> inputInfoEvents = iEvent.getEvents();
       try {
         if (inputInfoEvents != null && !inputInfoEvents.isEmpty()) {
-          vertex.handleRoutedTezEvents(inputInfoEvents, false, false);
+          vertex.initGeneratedEvents.addAll(inputInfoEvents);
+          vertex.handleRoutedTezEvents(inputInfoEvents, false);
         }
       } catch (AMUserCodeException e) {
         String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier();
@@ -3718,8 +2941,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
   public static class StartTransition implements
     MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
 
-  @Override
-  public VertexState transition(VertexImpl vertex, VertexEvent event) {
+    @Override
+    public VertexState transition(VertexImpl vertex, VertexEvent event) {
       Preconditions.checkState(vertex.getState() == VertexState.INITED,
           "Unexpected state " + vertex.getState() + " for " + vertex.logIdentifier);
       // if the start signal is pending this event is a fake start event to trigger this transition
@@ -3739,19 +2962,22 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       if (completelyConfiguredSent.compareAndSet(false, true)) {
         stateChangeNotifier.stateChanged(vertexId, new VertexStateUpdate(vertexName,
             org.apache.tez.dag.api.event.VertexState.CONFIGURED));
+        logVertexConfigurationDoneEvent();
       }
-    }    
+    }
   }
 
   private VertexState startVertex() {
-    // !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
-    // IMPORTANT - Until Recovery is fixed to use normal state transitions, if any code is added 
-    // here then please check if it needs to be duplicated in recoveryCodeSimulatingStart().
-    // !!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!
     Preconditions.checkState(getState() == VertexState.INITED,
         "Vertex must be inited " + logIdentifier);
 
-    startedTime = clock.getTime();
+    if (recoveryData != null && recoveryData.isVertexStarted()) {
+      VertexStartedEvent vertexStartedEvent = recoveryData.getVertexStartedEvent();
+      this.startedTime = vertexStartedEvent.getStartTime();
+    } else {
+      this.startedTime = clock.getTime();
+    }
+
     try {
       vertexManager.onVertexStarted(getTaskAttemptIdentifiers(dag, pendingReportedSrcCompletions));
     } catch (AMUserCodeException e) {
@@ -3967,14 +3193,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       String msg = "Exception in " + e.getSource() + ", vertex:" + vertex.getLogIdentifier();
       LOG.error(msg, e);
       
-      if (vertex.getState() == VertexState.RECOVERING) {
-        LOG.info("Received a user code error during recovering, setting recovered"
-            + " state to FAILED");
-        vertex.addDiagnostic(msg + "," + ExceptionUtils.getStackTrace(e.getCause()));
-        vertex.trySetTerminationCause(VertexTerminationCause.AM_USERCODE_FAILURE);
-        vertex.recoveredState = VertexState.FAILED;
-        return VertexState.RECOVERING;
-      } else if (vertex.getState() == VertexState.RUNNING || vertex.getState() == VertexState.COMMITTING) {
+      if (vertex.getState() == VertexState.RUNNING || vertex.getState() == VertexState.COMMITTING) {
         vertex.addDiagnostic(msg + "," + ExceptionUtils.getStackTrace(e.getCause()));
         vertex.tryEnactKill(VertexTerminationCause.AM_USERCODE_FAILURE,
             TaskTerminationCause.AM_USERCODE_FAILURE);
@@ -4269,10 +3488,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     @Override
     public VertexState transition(VertexImpl vertex, VertexEvent event) {
       VertexEventRouteEvent rEvent = (VertexEventRouteEvent) event;
-      boolean recovered = rEvent.isRecovered();
       List<TezEvent> tezEvents = rEvent.getEvents();
       try {
-        vertex.handleRoutedTezEvents(tezEvents, recovered, false);
+        vertex.handleRoutedTezEvents(tezEvents, false);
       } catch (AMUserCodeException e) {
         String msg = "Exception in " + e.getSource() + ", vertex=" + vertex.getLogIdentifier();
         LOG.error(msg, e);
@@ -4413,37 +3631,11 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
     return new TaskAttemptEventInfo(nextFromEventId, events, nextPreRoutedFromEventId);
   }
 
-  private void handleRoutedTezEvents(List<TezEvent> tezEvents, boolean recovered, boolean isPendingEvents) throws AMUserCodeException {
-    if (getAppContext().isRecoveryEnabled()
-        && !recovered
-        && !isPendingEvents
-        && !tezEvents.isEmpty()) {
-      List<TezEvent> recoveryEvents =
-          Lists.newArrayList();
-      for (TezEvent tezEvent : tezEvents) {
-        if (!isEventFromVertex(this, tezEvent.getSourceInfo())) {
-          continue;
-        }
-        if  (tezEvent.getEventType().equals(EventType.COMPOSITE_DATA_MOVEMENT_EVENT)
-          || tezEvent.getEventType().equals(EventType.DATA_MOVEMENT_EVENT)
-          || tezEvent.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)
-          || tezEvent.getEventType().equals(EventType.ROOT_INPUT_INITIALIZER_EVENT)) {
-          recoveryEvents.add(tezEvent);
-        }
-      }
-      if (!recoveryEvents.isEmpty()) {
-        VertexRecoverableEventsGeneratedEvent historyEvent =
-            new VertexRecoverableEventsGeneratedEvent(vertexId,
-                recoveryEvents);
-        appContext.getHistoryHandler().handle(
-            new DAGHistoryEvent(getDAGId(), historyEvent));
-      }
-    }
+  private void handleRoutedTezEvents(List<TezEvent> tezEvents, boolean isPendingEvents) throws AMUserCodeException {
     for(TezEvent tezEvent : tezEvents) {
       if (LOG.isDebugEnabled()) {
         LOG.debug("Vertex: " + getLogIdentifier() + " routing event: "
-            + tezEvent.getEventType()
-            + " Recovered:" + recovered);
+            + tezEvent.getEventType());
       }
       EventMetaData sourceMeta = tezEvent.getSourceInfo();
       switch(tezEvent.getEventType()) {
@@ -4580,44 +3772,6 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
           srcEdge.sendTezEventToSourceTasks(tezEvent);
         }
         break;
-      case TASK_ATTEMPT_FAILED_EVENT:
-        {
-          checkEventSourceMetadata(this, sourceMeta);
-          TaskAttemptTerminationCause errCause = null;
-          switch (sourceMeta.getEventGenerator()) {
-          case INPUT:
-            errCause = TaskAttemptTerminationCause.INPUT_READ_ERROR;
-            break;
-          case PROCESSOR:
-            errCause = TaskAttemptTerminationCause.APPLICATION_ERROR;
-            break;
-          case OUTPUT:
-            errCause = TaskAttemptTerminationCause.OUTPUT_WRITE_ERROR;
-            break;
-          case SYSTEM:
-            errCause = TaskAttemptTerminationCause.FRAMEWORK_ERROR;
-            break;
-          default:
-            throw new TezUncheckedException("Unknown EventProducerConsumerType: " +
-                sourceMeta.getEventGenerator());
-          }
-          TaskAttemptFailedEvent taskFailedEvent =
-              (TaskAttemptFailedEvent) tezEvent.getEvent();
-          getEventHandler().handle(
-              new TaskAttemptEventAttemptFailed(sourceMeta.getTaskAttemptID(),
-                  TaskAttemptEventType.TA_FAILED,
-                  "Error: " + taskFailedEvent.getDiagnostics(), 
-                  errCause)
-              );
-        }
-        break;
-      case TASK_ATTEMPT_COMPLETED_EVENT:
-        {
-          checkEventSourceMetadata(this, sourceMeta);
-          getEventHandler().handle(
-              new TaskAttemptEvent(sourceMeta.getTaskAttemptID(), TaskAttemptEventType.TA_DONE));
-        }
-        break;
       default:
         throw new TezUncheckedException("Unhandled tez event type: "
             + tezEvent.getEventType());
@@ -4708,12 +3862,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
           return org.apache.tez.dag.api.event.VertexState.FAILED;
         case KILLED:
           return org.apache.tez.dag.api.event.VertexState.KILLED;
-        case NEW:
         case INITIALIZING:
+          return org.apache.tez.dag.api.event.VertexState.INITIALIZING;
+        case NEW:
         case INITED:
         case ERROR:
         case TERMINATING:
-        case RECOVERING:
         default:
           throw new TezUncheckedException(
               "Not expecting state updates for state: " + vertexState + ", VertexID: " + vertexId);
@@ -5104,4 +4258,100 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex, EventHandl
       LOG.debug("Vertex: " + vertexName + ", rack: " + rack.toString());
     }
   }
+
+  /**
+   * This is for recovery when VertexReconfigureDoneEvent is seen. 
+   */
+  public static class NoOpVertexManager extends VertexManagerPlugin {
+
+    private VertexConfigurationDoneEvent configurationDoneEvent;
+    private boolean setParallelismInInitializing = false;
+
+    public NoOpVertexManager(VertexManagerPluginContext context) {
+      super(context);
+    }
+
+    @Override
+    public void initialize() throws Exception {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("initialize NoOpVertexManager");
+      }
+      configurationDoneEvent = new VertexConfigurationDoneEvent();
+      configurationDoneEvent.fromProtoStream(new ByteArrayInputStream(getContext().getUserPayload().deepCopyAsArray()));
+      String vertexName = getContext().getVertexName();
+      if (getContext().getVertexNumTasks(vertexName) == -1) {
+        Preconditions.checkArgument(configurationDoneEvent.isSetParallelismCalled(), "SetParallelism must be called "
+            + "when numTasks is -1");
+        setParallelismInInitializing = true;
+        getContext().registerForVertexStateUpdates(vertexName,
+            Sets.newHashSet(org.apache.tez.dag.api.event.VertexState.INITIALIZING));
+      }
+      getContext().vertexReconfigurationPlanned();
+    }
+
+    @Override
+    public void onVertexStarted(List<TaskAttemptIdentifier> completions)
+        throws Exception {
+      // apply the ReconfigureDoneEvent and then schedule all the tasks.
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("onVertexStarted is invoked in NoOpVertexManager, vertex=" + getContext().getVertexName());
+      }
+      if (!setParallelismInInitializing && configurationDoneEvent.isSetParallelismCalled()) {
+        reconfigureVertex();
+      }
+      getContext().doneReconfiguringVertex();
+      int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("Schedule all the tasks, numTask=" + numTasks);
+      }
+      List<ScheduleTaskRequest> tasks = new ArrayList<ScheduleTaskRequest>();
+      for (int i=0;i<numTasks;++i) {
+        tasks.add(ScheduleTaskRequest.create(i, null));
+      }
+      getContext().scheduleTasks(tasks);
+    }
+
+    @Override
+    public void onSourceTaskCompleted(TaskAttemptIdentifier attempt)
+        throws Exception {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("onSourceTaskCompleted is invoked in NoOpVertexManager, vertex=" + getContext().getVertexName());
+      }
+    }
+
+    @Override
+    public void onVertexManagerEventReceived(VertexManagerEvent vmEvent)
+        throws Exception {
+      if (LOG.isDebugEnabled()) {
+        LOG.debug("onVertexManagerEventReceived is invoked in NoOpVertexManager, vertex=" + getContext().getVertexName());
+      }
+    }
+
+    @Override
+    public

<TRUNCATED>

[3/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
deleted file mode 100644
index 1aba5fa..0000000
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
+++ /dev/null
@@ -1,327 +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.dag.impl;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.times;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.event.Event;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
-import org.apache.tez.dag.api.oldrecords.TaskState;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.ContainerContext;
-import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
-import org.apache.tez.dag.app.TaskHeartbeatHandler;
-import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
-import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
-import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate;
-import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl.DataEventDependencyInfo;
-import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.HistoryEventHandler;
-import org.apache.tez.dag.history.HistoryEventType;
-import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
-import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
-import org.apache.tez.dag.history.events.TaskFinishedEvent;
-import org.apache.tez.dag.records.TaskAttemptTerminationCause;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.dag.records.TezVertexID;
-import org.junit.Before;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-
-import com.google.common.collect.Lists;
-
-@SuppressWarnings({ "unchecked", "rawtypes" })
-public class TestTaskAttemptRecovery {
-
-  private TaskAttemptImpl ta;
-  private EventHandler mockEventHandler;
-  private long creationTime = System.currentTimeMillis();
-  private long allocationTime = creationTime + 5000;
-  private long startTime = allocationTime + 5000;
-  private long finishTime = startTime + 5000;
-
-  private TezTaskAttemptID taId;
-  private String vertexName = "v1";
-
-  private AppContext mockAppContext;
-  private MockHistoryEventHandler mockHistoryEventHandler;
-  private Task mockTask;
-  private Vertex mockVertex;
-
-  public static class MockHistoryEventHandler extends HistoryEventHandler {
-
-    private List<DAGHistoryEvent> events;
-
-    public MockHistoryEventHandler(AppContext context) {
-      super(context);
-      events = new ArrayList<DAGHistoryEvent>();
-    }
-
-    @Override
-    public void handle(DAGHistoryEvent event) {
-      events.add(event);
-    }
-
-    @Override
-    public void handleCriticalEvent(DAGHistoryEvent event) throws IOException {
-      events.add(event);
-    }
-
-    void verfiyTaskAttemptFinishedEvent(TezTaskAttemptID taId, TaskAttemptState finalState, int expectedTimes) {
-      int actualTimes = 0;
-      for (DAGHistoryEvent event : events) {
-        if (event.getHistoryEvent().getEventType() == HistoryEventType.TASK_ATTEMPT_FINISHED) {
-          TaskAttemptFinishedEvent tfEvent = (TaskAttemptFinishedEvent)event.getHistoryEvent();
-          if (tfEvent.getTaskAttemptID().equals(taId) &&
-              tfEvent.getState().equals(finalState)) {
-            actualTimes ++;
-          }
-        }
-      }
-      assertEquals(expectedTimes, actualTimes);
-    }
-
-    void verifyTaskFinishedEvent(TezTaskID taskId, TaskState finalState, int expectedTimes) {
-      int actualTimes = 0;
-      for (DAGHistoryEvent event : events) {
-        if (event.getHistoryEvent().getEventType() == HistoryEventType.TASK_FINISHED) {
-          TaskFinishedEvent tfEvent = (TaskFinishedEvent)event.getHistoryEvent();
-          if (tfEvent.getTaskID().equals(taskId) && tfEvent.getState().equals(finalState)) {
-            actualTimes ++;
-          }
-        }
-      }
-      assertEquals(expectedTimes, actualTimes);
-    }
-  }
-
-  @Before
-  public void setUp() {
-    mockTask = mock(Task.class);
-    mockVertex = mock(Vertex.class);
-    when(mockTask.getVertex()).thenReturn(mockVertex);
-    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
-    when(mockAppContext.getCurrentDAG().getVertex(any(TezVertexID.class))
-      .getTask(any(TezTaskID.class)))
-      .thenReturn(mockTask);
-    mockHistoryEventHandler = new MockHistoryEventHandler(mockAppContext);
-    when(mockAppContext.getHistoryHandler()).thenReturn(mockHistoryEventHandler);
-    mockEventHandler = mock(EventHandler.class);
-    TezTaskID taskId =
-        TezTaskID.fromString("task_1407371892933_0001_1_00_000000");
-    ta =
-        new TaskAttemptImpl(taskId, 0, mockEventHandler,
-            mock(TaskCommunicatorManagerInterface.class), new Configuration(),
-            new SystemClock(), mock(TaskHeartbeatHandler.class),
-            mockAppContext, false, Resource.newInstance(1, 1),
-            mock(ContainerContext.class), false, mockTask);
-    taId = ta.getID();
-  }
-
-  private void restoreFromTAStartEvent() {
-    TaskAttemptState recoveredState =
-        ta.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
-            startTime, mock(ContainerId.class), mock(NodeId.class), "", "", ""));
-    assertEquals(startTime, ta.getLaunchTime());
-    assertEquals(TaskAttemptState.RUNNING, recoveredState);
-  }
-
-  private void restoreFromTAFinishedEvent(TaskAttemptState state) {
-    String diag = "test_diag";
-    TezCounters counters = mock(TezCounters.class);
-    TezTaskAttemptID causalId = TezTaskAttemptID.getInstance(taId.getTaskID(), taId.getId()+1);
-    
-    TaskAttemptTerminationCause errorEnum = null;
-    if (state != TaskAttemptState.SUCCEEDED) {
-      errorEnum = TaskAttemptTerminationCause.APPLICATION_ERROR;
-    }
-
-    long lastDataEventTime = 1024;
-    TezTaskAttemptID lastDataEventTA = mock(TezTaskAttemptID.class);
-    List<DataEventDependencyInfo> events = Lists.newLinkedList();
-    events.add(new DataEventDependencyInfo(lastDataEventTime, lastDataEventTA));
-    events.add(new DataEventDependencyInfo(lastDataEventTime, lastDataEventTA));
-    TaskAttemptState recoveredState =
-        ta.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            startTime, finishTime, state, errorEnum, diag, counters, events, creationTime,
-            causalId, allocationTime));
-    assertEquals(causalId, ta.getCreationCausalAttempt());
-    assertEquals(creationTime, ta.getCreationTime());
-    assertEquals(allocationTime, ta.getAllocationTime());
-    assertEquals(startTime, ta.getLaunchTime());
-    assertEquals(finishTime, ta.getFinishTime());
-    assertEquals(counters, ta.reportedStatus.counters);
-    assertEquals(1.0f, ta.reportedStatus.progress, 1e-6);
-    assertEquals(state, ta.reportedStatus.state);
-    assertEquals(1, ta.getDiagnostics().size());
-    assertEquals(diag, ta.getDiagnostics().get(0));
-    assertEquals(state, recoveredState);
-    assertEquals(events.size(), ta.lastDataEvents.size());
-    assertEquals(lastDataEventTime, ta.lastDataEvents.get(0).getTimestamp());
-    assertEquals(lastDataEventTA, ta.lastDataEvents.get(0).getTaskAttemptId());
-    if (state != TaskAttemptState.SUCCEEDED) {
-      assertEquals(errorEnum, ta.getTerminationCause());
-    } else {
-      assertEquals(TaskAttemptTerminationCause.UNKNOWN_ERROR, ta.getTerminationCause());
-    }
-  }
-
-  private void verifyEvents(List<Event> events, Class<? extends Event> eventClass,
-      int expectedTimes) {
-    int actualTimes = 0;
-    for (Event event : events) {
-      if (eventClass.isInstance(event)) {
-        actualTimes ++;
-      }
-    }
-    assertEquals(expectedTimes, actualTimes);
-  }
-
-  /**
-   * No any event to restore -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testTARecovery_NEW() {
-    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
-    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
-
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    assertEquals(2, events.size());
-    verifyEvents(events, TaskEventTAUpdate.class, 1);
-    // one for task killed
-    verifyEvents(events, DAGEventCounterUpdate.class, 1);
-  }
-
-  /**
-   * restoreFromTAStartEvent -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testTARecovery_START() {
-    restoreFromTAStartEvent();
-
-    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
-    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
-
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(3)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    assertEquals(3, events.size());
-    verifyEvents(events, TaskEventTAUpdate.class, 1);
-    // one for task launch, one for task killed
-    verifyEvents(events, DAGEventCounterUpdate.class, 2);
-
-    mockHistoryEventHandler.verfiyTaskAttemptFinishedEvent(taId, TaskAttemptState.KILLED, 1);
-  }
-
-  /**
-   * restoreFromTAStartEvent -> restoreFromTAFinished (SUCCEED)
-   * -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testTARecovery_SUCCEED() {
-    restoreFromTAStartEvent();
-    restoreFromTAFinishedEvent(TaskAttemptState.SUCCEEDED);
-
-    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
-    assertEquals(TaskAttemptStateInternal.SUCCEEDED, ta.getInternalState());
-
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    assertEquals(2, events.size());
-    // one for task launch, one for task succeeded
-    verifyEvents(events, DAGEventCounterUpdate.class, 2);
-  }
-
-  /**
-   * restoreFromTAStartEvent -> restoreFromTAFinished (KILLED)
-   * -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testTARecovery_KIILED() {
-    restoreFromTAStartEvent();
-    restoreFromTAFinishedEvent(TaskAttemptState.KILLED);
-
-    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
-    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
-
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    assertEquals(2, events.size());
-    // one for task launch, one for task killed
-    verifyEvents(events, DAGEventCounterUpdate.class, 2);
-  }
-
-  /**
-   * restoreFromTAStartEvent -> restoreFromTAFinished (FAILED)
-   * -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testTARecovery_FAILED() {
-    restoreFromTAStartEvent();
-    restoreFromTAFinishedEvent(TaskAttemptState.FAILED);
-
-    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
-    assertEquals(TaskAttemptStateInternal.FAILED, ta.getInternalState());
-
-    ArgumentCaptor<Event> eventCaptor = ArgumentCaptor.forClass(Event.class);
-    verify(mockEventHandler, times(2)).handle(eventCaptor.capture());
-    List<Event> events = eventCaptor.getAllValues();
-    assertEquals(2, events.size());
-    // one for task launch, one for task killed
-    verifyEvents(events, DAGEventCounterUpdate.class, 2);
-  }
-
-  /**
-   * restoreFromTAFinishedEvent ( killed before started)
-   */
-  @Test(timeout = 5000)
-  public void testRecover_FINISH_BUT_NO_START() {
-    TaskAttemptState recoveredState =
-        ta.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            startTime, finishTime, TaskAttemptState.KILLED,
-            TaskAttemptTerminationCause.APPLICATION_ERROR, "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskAttemptState.KILLED, recoveredState);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
index 24c9664..0414c99 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskImpl.java
@@ -163,7 +163,7 @@ public class TestTaskImpl {
   }
 
   private void scheduleTaskAttempt(TezTaskID taskId) {
-    mockTask.handle(new TaskEventScheduleTask(taskId, mockTaskSpec, locationHint));
+    mockTask.handle(new TaskEventScheduleTask(taskId, mockTaskSpec, locationHint, false));
     assertTaskScheduledState();
     assertEquals(mockTaskSpec, mockTask.getBaseTaskSpec());
     assertEquals(locationHint, mockTask.getTaskLocationHint());
@@ -762,8 +762,7 @@ public class TestTaskImpl {
         boolean isRescheduled,
         Resource resource, ContainerContext containerContext, TezTaskAttemptID schedCausalTA) {
       super(taskId, attemptNumber, eventHandler, tal, conf, clock, thh,
-          appContext, isRescheduled, resource, containerContext, false, mock(TaskImpl.class),
-          schedCausalTA);
+          appContext, isRescheduled, resource, containerContext, false, mock(TaskImpl.class), schedCausalTA);
     }
     
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
deleted file mode 100644
index bea423a..0000000
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskRecovery.java
+++ /dev/null
@@ -1,873 +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.dag.impl;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
-import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.NodeId;
-import org.apache.hadoop.yarn.api.records.Resource;
-import org.apache.hadoop.yarn.event.DrainDispatcher;
-import org.apache.hadoop.yarn.event.EventHandler;
-import org.apache.hadoop.yarn.util.SystemClock;
-import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezUncheckedException;
-import org.apache.tez.dag.api.client.VertexStatus.State;
-import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
-import org.apache.tez.dag.api.oldrecords.TaskState;
-import org.apache.tez.dag.app.AppContext;
-import org.apache.tez.dag.app.ContainerContext;
-import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
-import org.apache.tez.dag.app.TaskHeartbeatHandler;
-import org.apache.tez.dag.app.dag.StateChangeNotifier;
-import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
-import org.apache.tez.dag.app.dag.TaskStateInternal;
-import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.dag.app.dag.event.DAGEventType;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
-import org.apache.tez.dag.app.dag.event.TaskEvent;
-import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
-import org.apache.tez.dag.app.dag.event.TaskEventType;
-import org.apache.tez.dag.app.dag.event.VertexEventType;
-import org.apache.tez.dag.app.dag.impl.TestTaskAttemptRecovery.MockHistoryEventHandler;
-import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
-import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
-import org.apache.tez.dag.history.events.TaskFinishedEvent;
-import org.apache.tez.dag.history.events.TaskStartedEvent;
-import org.apache.tez.dag.records.TaskAttemptTerminationCause;
-import org.apache.tez.dag.records.TezDAGID;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.dag.records.TezTaskID;
-import org.apache.tez.dag.records.TezVertexID;
-import org.apache.tez.runtime.api.OutputCommitter;
-import org.apache.tez.runtime.api.OutputCommitterContext;
-import org.junit.Before;
-import org.junit.Test;
-
-import com.google.common.collect.Lists;
-
-public class TestTaskRecovery {
-
-  private TaskImpl task;
-  private DrainDispatcher dispatcher;
-
-  private int taskAttemptCounter = 0;
-
-  private Configuration conf = new Configuration();
-  private AppContext mockAppContext;
-  private MockHistoryEventHandler  mockHistoryEventHandler;
-  private ApplicationId appId = ApplicationId.newInstance(
-      System.currentTimeMillis(), 1);
-  private TezDAGID dagId = TezDAGID.getInstance(appId, 1);
-  private TezVertexID vertexId = TezVertexID.getInstance(dagId, 1);
-  private Vertex vertex;
-  private String vertexName = "v1";
-  private long taskScheduledTime = 100L;
-  private long taskStartTime = taskScheduledTime + 100L;
-  private long taskFinishTime = taskStartTime + 100L;
-  private TaskAttemptEventHandler taEventHandler =
-      new TaskAttemptEventHandler();
-
-  private class TaskEventHandler implements EventHandler<TaskEvent> {
-    @Override
-    public void handle(TaskEvent event) {
-      task.handle(event);
-    }
-  }
-
-  private class TaskAttemptEventHandler implements
-      EventHandler<TaskAttemptEvent> {
-
-    private List<TaskAttemptEvent> events = Lists.newArrayList();
-
-    @Override
-    public void handle(TaskAttemptEvent event) {
-      events.add(event);
-      ((TaskAttemptImpl) task.getAttempt(event.getTaskAttemptID()))
-          .handle(event);
-    }
-
-    public List<TaskAttemptEvent> getEvents() {
-      return events;
-    }
-  }
-
-  private class TestOutputCommitter extends OutputCommitter {
-
-    boolean recoverySupported = false;
-    boolean throwExceptionWhenRecovery = false;
-
-    public TestOutputCommitter(OutputCommitterContext committerContext,
-        boolean recoverySupported, boolean throwExceptionWhenRecovery) {
-      super(committerContext);
-      this.recoverySupported = recoverySupported;
-      this.throwExceptionWhenRecovery = throwExceptionWhenRecovery;
-    }
-
-    @Override
-    public void recoverTask(int taskIndex, int previousDAGAttempt)
-        throws Exception {
-      if (throwExceptionWhenRecovery) {
-        throw new Exception("fail recovery Task");
-      }
-    }
-
-    @Override
-    public boolean isTaskRecoverySupported() {
-      return recoverySupported;
-    }
-
-    @Override
-    public void initialize() throws Exception {
-
-    }
-
-    @Override
-    public void setupOutput() throws Exception {
-
-    }
-
-    @Override
-    public void commitOutput() throws Exception {
-
-    }
-
-    @Override
-    public void abortOutput(State finalState) throws Exception {
-
-    }
-
-  }
-
-  @Before
-  public void setUp() {
-    dispatcher = new DrainDispatcher();
-    dispatcher.register(DAGEventType.class, mock(EventHandler.class));
-    dispatcher.register(VertexEventType.class, mock(EventHandler.class));
-    dispatcher.register(TaskEventType.class, new TaskEventHandler());
-    dispatcher.register(TaskAttemptEventType.class, taEventHandler);
-    dispatcher.init(new Configuration());
-    dispatcher.start();
-
-    vertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
-    when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
-
-    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
-    when(mockAppContext.getCurrentDAG().getVertex(any(TezVertexID.class)))
-        .thenReturn(vertex);
-    mockHistoryEventHandler = new MockHistoryEventHandler(mockAppContext);
-    when(mockAppContext.getHistoryHandler()).thenReturn(mockHistoryEventHandler);
-    task =
-        new TaskImpl(vertexId, 0, dispatcher.getEventHandler(),
-            new Configuration(), mock(TaskCommunicatorManagerInterface.class),
-            new SystemClock(), mock(TaskHeartbeatHandler.class),
-            mockAppContext, false, Resource.newInstance(1, 1),
-            mock(ContainerContext.class), mock(StateChangeNotifier.class), vertex);
-
-    Map<String, OutputCommitter> committers =
-        new HashMap<String, OutputCommitter>();
-    committers.put("out1", new TestOutputCommitter(
-        mock(OutputCommitterContext.class), true, false));
-    when(task.getVertex().getOutputCommitters()).thenReturn(committers);
-  }
-
-  private void restoreFromTaskStartEvent() {
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskStartedEvent(task.getTaskId(),
-            vertexName, taskScheduledTime, taskStartTime));
-    assertEquals(TaskState.SCHEDULED, recoveredState);
-    assertEquals(0, task.getFinishedAttemptsCount());
-    assertEquals(taskScheduledTime, task.scheduledTime);
-    assertEquals(0, task.getAttempts().size());
-  }
-
-  private void restoreFromFirstTaskAttemptStartEvent(TezTaskAttemptID taId) {
-    long taStartTime = taskStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
-            taStartTime, mock(ContainerId.class), mock(NodeId.class), "", "", ""));
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(0, task.getFinishedAttemptsCount());
-    assertEquals(taskScheduledTime, task.scheduledTime);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(TaskAttemptStateInternal.NEW,
-        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
-    assertEquals(1, task.getUncompletedAttemptsCount());
-  }
-
-  /**
-   * New -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_New() {
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.NEW, task.getInternalState());
-  }
-
-  /**
-   * -> restoreFromTaskFinishEvent ( no TaskStartEvent )
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_NoStartEvent() {
-    try {
-      task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(), vertexName,
-          taskStartTime, taskFinishTime, null, TaskState.SUCCEEDED, "",
-          new TezCounters(), 0));
-      fail("Should fail due to no TaskStartEvent before TaskFinishEvent");
-    } catch (Throwable e) {
-      assertTrue(e.getMessage().contains(
-          "Finished Event seen but"
-              + " no Started Event was encountered earlier"));
-    }
-  }
-
-  /**
-   * -> restoreFromTaskFinishEvent ( no TaskStartEvent )
-   */
-  @Test(timeout = 5000)
-  public void testRecoveryNewToKilled_NoStartEvent() {
-    task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(), vertexName,
-        taskStartTime, taskFinishTime, null, TaskState.KILLED, "",
-        new TezCounters(), 0));
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Started() {
-    restoreFromTaskStartEvent();
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // new task attempt is scheduled
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(0, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptFinishedEvent (KILLED) ->
-   * RecoverTranstion
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OnlyTAFinishedEvent_KILLED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-        0L, 0L, TaskAttemptState.KILLED, TaskAttemptTerminationCause.TERMINATED_BY_CLIENT,"", new TezCounters(), null, 0, null, 0));
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    // wait for the second task attempt is scheduled
-    dispatcher.await();
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // taskAttempt_1 is recovered to KILLED, and new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptFinishedEvent (FAILED) ->
-   * RecoverTranstion
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OnlyTAFinishedEvent_FAILED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-        0L, 0L, TaskAttemptState.FAILED, TaskAttemptTerminationCause.CONTAINER_LAUNCH_FAILED,"", new TezCounters(), null, 0, null, 0));
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    // wait for the second task attempt is scheduled
-    dispatcher.await();
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // taskAttempt_1 is recovered to FAILED, and new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(1, task.failedAttempts);
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptFinishedEvent (SUCCEEDED) ->
-   * RecoverTranstion
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OnlyTAFinishedEvent_SUCCEEDED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    try {
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-          0L, 0L, TaskAttemptState.SUCCEEDED, null ,"", new TezCounters(), null, 0, null, 0));
-      fail("Should fail due to no TaskAttemptStartedEvent but with TaskAttemptFinishedEvent(Succeeded)");
-    } catch (TezUncheckedException e) {
-      assertTrue(e.getMessage().contains("Could not find task attempt when trying to recover"));
-    }
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * RecoverTranstion
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    // wait for the second task attempt is scheduled
-    dispatcher.await();
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // taskAttempt_1 is recovered to KILLED, and new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted_SUCCEEDED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.SUCCEEDED, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-    mockHistoryEventHandler.verifyTaskFinishedEvent(task.getTaskId(), TaskState.SUCCEEDED, 1);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (FAILED) -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted_FAILED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.FAILED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(1, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(1, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (KILLED) -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted_KILLED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.KILLED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) ->
-   * restoreFromTaskFinishedEvent -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted_SUCCEEDED_Finished() {
-
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.SUCCEEDED, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-
-    recoveredState =
-        task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(),
-            vertexName, taskStartTime, taskFinishTime, taId,
-            TaskState.SUCCEEDED, "", new TezCounters(), 0));
-    assertEquals(TaskState.SUCCEEDED, recoveredState);
-    assertEquals(taId, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-    mockHistoryEventHandler.verifyTaskFinishedEvent(task.getTaskId(), TaskState.SUCCEEDED, 1);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) ->
-   * restoreFromTaskAttemptFinishedEvent (Failed due to output_failure)
-   * restoreFromTaskFinishedEvent -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted_SUCCEEDED_FAILED() {
-
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.SUCCEEDED, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-
-    // it is possible for TaskAttempt transit from SUCCEEDED to FAILURE due to output failure.
-    recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.FAILED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(1, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(1, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) ->
-   * restoreFromTaskAttemptFinishedEvent (KILLED due to node failed )
-   * restoreFromTaskFinishedEvent -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted_SUCCEEDED_KILLED() {
-
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.SUCCEEDED, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-
-    // it is possible for TaskAttempt transit from SUCCEEDED to KILLED due to node failure.
-    recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.KILLED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Commit_Failed_Recovery_Not_Supported() {
-    Map<String, OutputCommitter> committers =
-        new HashMap<String, OutputCommitter>();
-    committers.put("out1", new TestOutputCommitter(
-        mock(OutputCommitterContext.class), false, false));
-    when(task.getVertex().getOutputCommitters()).thenReturn(committers);
-
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    // restoreFromTaskAttemptFinishedEvent (SUCCEEDED)
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.SUCCEEDED, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_Commit_Failed_recover_fail() {
-    Map<String, OutputCommitter> committers =
-        new HashMap<String, OutputCommitter>();
-    committers.put("out1", new TestOutputCommitter(
-        mock(OutputCommitterContext.class), true, true));
-    when(task.getVertex().getOutputCommitters()).thenReturn(committers);
-
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-
-    // restoreFromTaskAttemptFinishedEvent (SUCCEEDED)
-    long taStartTime = taskStartTime + 100L;
-    long taFinishTime = taStartTime + 100L;
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.SUCCEEDED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.SUCCEEDED, recoveredState);
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(taId, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  @Test(timeout = 5000)
-  public void testRecovery_WithDesired_SUCCEEDED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-    task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.SUCCEEDED,
-        false));
-    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
-    // no TA_Recovery event sent
-    assertEquals(0, taEventHandler.getEvents().size());
-  }
-
-  @Test(timeout = 5000)
-  public void testRecovery_WithDesired_FAILED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-    task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.FAILED,
-        false));
-    assertEquals(TaskStateInternal.FAILED, task.getInternalState());
-    // no TA_Recovery event sent
-    assertEquals(0, taEventHandler.getEvents().size());
-  }
-
-  @Test(timeout = 5000)
-  public void testRecovery_WithDesired_KILLED() {
-    restoreFromTaskStartEvent();
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    restoreFromFirstTaskAttemptStartEvent(taId);
-    task.handle(new TaskEventRecoverTask(task.getTaskId(), TaskState.KILLED,
-        false));
-    assertEquals(TaskStateInternal.KILLED, task.getInternalState());
-    // no TA_Recovery event sent
-    assertEquals(0, taEventHandler.getEvents().size());
-
-  }
-
-  /**
-   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
-   * restoreFromTaskAttemptFinishedEvent (KILLED) -> RecoverTransition
-   */
-  @Test(timeout = 5000)
-  public void testRecovery_OneTAStarted_Killed() {
-    restoreFromTaskStartEvent();
-
-    long taStartTime = taskStartTime + 100L;
-    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName,
-            taStartTime, mock(ContainerId.class), mock(NodeId.class), "", "", ""));
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(TaskAttemptStateInternal.NEW,
-        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(0, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-
-    long taFinishTime = taStartTime + 100L;
-    recoveredState =
-        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
-            taStartTime, taFinishTime, TaskAttemptState.KILLED, null,
-            "", new TezCounters(), null, 0, null, 0));
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(TaskAttemptStateInternal.NEW,
-        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
-    assertEquals(1, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(0, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    // wait for Task send TA_RECOVER to TA and TA complete the RecoverTransition
-    dispatcher.await();
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    assertEquals(TaskAttemptStateInternal.KILLED,
-        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
-    // new task attempt is scheduled
-    assertEquals(2, task.getAttempts().size());
-    assertEquals(1, task.getFinishedAttemptsCount());
-    assertEquals(0, task.failedAttempts);
-    assertEquals(1, task.getUncompletedAttemptsCount());
-    assertEquals(null, task.successfulAttempt);
-  }
-
-  /**
-   * n = maxFailedAttempts, in the previous AM attempt, n task attempts are
-   * killed. When recovering, it should continue to be in running state and
-   * schedule a new task attempt.
-   */
-  @Test(timeout = 5000)
-  public void testTaskRecovery_MultipleAttempts1() {
-    int maxFailedAttempts =
-        conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
-            TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    restoreFromTaskStartEvent();
-
-    for (int i = 0; i < maxFailedAttempts; ++i) {
-      TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-      task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
-          mock(ContainerId.class), mock(NodeId.class), "", "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
-          0, TaskAttemptState.KILLED, null, "", null, null, 0, null, 0));
-    }
-    assertEquals(maxFailedAttempts, task.getAttempts().size());
-    assertEquals(0, task.failedAttempts);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    // if the previous task attempt is killed, it should not been take into
-    // account when checking whether exceed the max attempts
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    // schedule a new task attempt
-    assertEquals(maxFailedAttempts + 1, task.getAttempts().size());
-  }
-
-  /**
-   * n = maxFailedAttempts, in the previous AM attempt, n task attempts are
-   * failed. When recovering, it should transit to failed because # of
-   * failed_attempt is exceeded.
-   */
-  @Test(timeout = 5000)
-  public void testTaskRecovery_MultipleAttempts2() {
-    int maxFailedAttempts =
-        conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
-            TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    restoreFromTaskStartEvent();
-
-    for (int i = 0; i < maxFailedAttempts; ++i) {
-      TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-      task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
-          mock(ContainerId.class), mock(NodeId.class), "", "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
-          0, TaskAttemptState.FAILED, null, "", null, null, 0, null, 0));
-    }
-    assertEquals(maxFailedAttempts, task.getAttempts().size());
-    assertEquals(maxFailedAttempts, task.failedAttempts);
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    // it should transit to failed because of the failed task attempt in the
-    // last application attempt.
-    assertEquals(TaskStateInternal.FAILED, task.getInternalState());
-    assertEquals(maxFailedAttempts, task.getAttempts().size());
-  }
-
-  /**
-   * n = maxFailedAttempts, in the previous AM attempt, n-1 task attempts are
-   * killed. And last task attempt is still in running state. When recovering,
-   * the last attempt should transit to killed and task is still in running
-   * state and new task attempt is scheduled.
-   */
-  @Test(timeout = 5000)
-  public void testTaskRecovery_MultipleAttempts3() throws InterruptedException {
-    int maxFailedAttempts =
-        conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
-            TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    restoreFromTaskStartEvent();
-
-    for (int i = 0; i < maxFailedAttempts - 1; ++i) {
-      TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
-      task.restoreFromEvent(new TaskAttemptStartedEvent(taId, vertexName, 0L,
-          mock(ContainerId.class), mock(NodeId.class), "", "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName, 0,
-          0, TaskAttemptState.FAILED, null, "", null, null, 0, null, 0));
-    }
-    assertEquals(maxFailedAttempts - 1, task.getAttempts().size());
-    assertEquals(maxFailedAttempts - 1, task.failedAttempts);
-
-    TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(task.getTaskId());
-    TaskState recoveredState =
-        task.restoreFromEvent(new TaskAttemptStartedEvent(newTaskAttemptId,
-            vertexName, 0, mock(ContainerId.class), mock(NodeId.class), "", "", ""));
-
-    assertEquals(TaskState.RUNNING, recoveredState);
-    assertEquals(TaskAttemptStateInternal.NEW,
-        ((TaskAttemptImpl) task.getAttempt(newTaskAttemptId))
-            .getInternalState());
-    assertEquals(maxFailedAttempts, task.getAttempts().size());
-
-    task.handle(new TaskEventRecoverTask(task.getTaskId()));
-    // wait until task attempt receive the Recover event from task
-    dispatcher.await();
-
-    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
-    assertEquals(TaskAttemptStateInternal.KILLED,
-        ((TaskAttemptImpl) (task.getAttempt(newTaskAttemptId)))
-            .getInternalState());
-    assertEquals(maxFailedAttempts - 1, task.failedAttempts);
-
-    // new task attempt is added
-    assertEquals(maxFailedAttempts + 1, task.getAttempts().size());
-  }
-
-  private TezTaskAttemptID getNewTaskAttemptID(TezTaskID taskId) {
-    return TezTaskAttemptID.getInstance(taskId, taskAttemptCounter++);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
index 035de32..11c2bf1 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexImpl.java
@@ -137,8 +137,10 @@ import org.apache.tez.dag.app.dag.event.CallableEventType;
 import org.apache.tez.dag.app.dag.event.DAGEvent;
 import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventSchedule;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventTerminationCauseEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.TaskEvent;
 import org.apache.tez.dag.app.dag.event.TaskEventScheduleTask;
@@ -3459,11 +3461,8 @@ public class TestVertexImpl {
 
     ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
     Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState());
-
-    dispatcher.getEventHandler().handle(
-        new VertexEventRouteEvent(v.getVertexId(), Collections.singletonList(new TezEvent(
-            new TaskAttemptFailedEvent("Failed"), new EventMetaData(
-                EventProducerConsumerType.PROCESSOR, v.getName(), null, ta.getID())))));
+    ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED,
+        "diag", TaskAttemptTerminationCause.APPLICATION_ERROR));
     dispatcher.await();
     Assert.assertEquals(VertexState.RUNNING, v.getState());
     Assert.assertEquals(TaskAttemptTerminationCause.APPLICATION_ERROR, ta.getTerminationCause());
@@ -3496,10 +3495,8 @@ public class TestVertexImpl {
     ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
     Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState());
 
-    dispatcher.getEventHandler().handle(
-        new VertexEventRouteEvent(v.getVertexId(), Collections.singletonList(new TezEvent(
-            new TaskAttemptFailedEvent("Failed"), new EventMetaData(
-                EventProducerConsumerType.INPUT, v.getName(), null, ta.getID())))));
+    ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED,
+        "diag", TaskAttemptTerminationCause.INPUT_READ_ERROR));
     dispatcher.await();
     Assert.assertEquals(VertexState.RUNNING, v.getState());
     Assert.assertEquals(TaskAttemptTerminationCause.INPUT_READ_ERROR, ta.getTerminationCause());
@@ -3533,10 +3530,8 @@ public class TestVertexImpl {
     ta.handle(new TaskAttemptEventStartedRemotely(ta.getID(), contId, null));
     Assert.assertEquals(TaskAttemptStateInternal.RUNNING, ta.getInternalState());
 
-    dispatcher.getEventHandler().handle(
-        new VertexEventRouteEvent(v.getVertexId(), Collections.singletonList(new TezEvent(
-            new TaskAttemptFailedEvent("Failed"), new EventMetaData(
-                EventProducerConsumerType.OUTPUT, v.getName(), null, ta.getID())))));
+    ta.handle(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED,
+        "diag", TaskAttemptTerminationCause.OUTPUT_WRITE_ERROR));
     dispatcher.await();
     Assert.assertEquals(VertexState.RUNNING, v.getState());
     Assert.assertEquals(TaskAttemptTerminationCause.OUTPUT_WRITE_ERROR, ta.getTerminationCause());
@@ -6355,17 +6350,17 @@ public class TestVertexImpl {
     v1.handle(new VertexEventRouteEvent(v1.getVertexId(), Lists.newArrayList(tezEvent1)));
     dispatcher.await();
     assertTrue(v3.pendingTaskEvents.size() != 0);
-    ArgumentCaptor<DAGHistoryEvent> argCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class);
-    verify(historyEventHandler, atLeast(1)).handle(argCaptor.capture());
-    verifyHistoryEvents(argCaptor.getAllValues(), HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED, 1);
+//    ArgumentCaptor<DAGHistoryEvent> argCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class);
+//    verify(historyEventHandler, atLeast(1)).handle(argCaptor.capture());
+//    verifyHistoryEvents(argCaptor.getAllValues(), HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED, 1);
 
     v3.scheduleTasks(Lists.newArrayList(ScheduleTaskRequest.create(0, null)));
     dispatcher.await();
     assertTrue(v3.pendingTaskEvents.size() == 0);
     // recovery events is not only handled one time
-    argCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class);
-    verify(historyEventHandler, atLeast(1)).handle(argCaptor.capture());
-    verifyHistoryEvents(argCaptor.getAllValues(), HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED, 1);
+//    argCaptor = ArgumentCaptor.forClass(DAGHistoryEvent.class);
+//    verify(historyEventHandler, atLeast(1)).handle(argCaptor.capture());
+//    verifyHistoryEvents(argCaptor.getAllValues(), HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED, 1);
   }
 
   private void verifyHistoryEvents(List<DAGHistoryEvent> events, HistoryEventType eventType, int expectedTimes) {


[8/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventRecoverTask.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventRecoverTask.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventRecoverTask.java
deleted file mode 100644
index f275a56..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventRecoverTask.java
+++ /dev/null
@@ -1,53 +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.dag.event;
-
-import org.apache.tez.dag.api.oldrecords.TaskState;
-import org.apache.tez.dag.records.TezTaskID;
-
-public class TaskEventRecoverTask extends TaskEvent {
-
-  TaskState desiredState;
-
-  boolean recoverDataForAttempts;
-
-  public TaskEventRecoverTask(TezTaskID taskID, TaskState desiredState) {
-    this(taskID, desiredState, true);
-  }
-
-  public TaskEventRecoverTask(TezTaskID taskID, TaskState desiredState,
-      boolean recoverData) {
-    super(taskID, TaskEventType.T_RECOVER);
-    this.desiredState = desiredState;
-    this.recoverDataForAttempts = recoverData;
-  }
-
-  public TaskEventRecoverTask(TezTaskID taskID) {
-    this(taskID, null);
-  }
-
-  public TaskState getDesiredState() {
-    return desiredState;
-  }
-
-  public boolean recoverData() {
-    return recoverDataForAttempts;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventScheduleTask.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventScheduleTask.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventScheduleTask.java
index 696602a..70d6043 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventScheduleTask.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventScheduleTask.java
@@ -22,14 +22,17 @@ import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.runtime.api.impl.TaskSpec;
 
-public class TaskEventScheduleTask extends TaskEvent {
+public class TaskEventScheduleTask extends TaskEvent implements RecoveryEvent {
   private final TaskSpec baseTaskSpec;
   private final TaskLocationHint locationHint;
-  
-  public TaskEventScheduleTask(TezTaskID taskId, TaskSpec baseTaskSpec, TaskLocationHint locationHint) {
+  private final boolean fromRecovery;
+
+  public TaskEventScheduleTask(TezTaskID taskId, TaskSpec baseTaskSpec, TaskLocationHint locationHint,
+      boolean fromRecovery) {
     super(taskId, TaskEventType.T_SCHEDULE);
     this.baseTaskSpec = baseTaskSpec;
     this.locationHint = locationHint;
+    this.fromRecovery = fromRecovery;
   }
   
   public TaskSpec getBaseTaskSpec() {
@@ -39,4 +42,9 @@ public class TaskEventScheduleTask extends TaskEvent {
   public TaskLocationHint getTaskLocationHint() {
     return locationHint;
   }
+
+  @Override
+  public boolean isFromRecovery() {
+    return fromRecovery;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTermination.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTermination.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTermination.java
index d48a0bf..1605869 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTermination.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventTermination.java
@@ -22,11 +22,12 @@ import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezTaskID;
 
 public class TaskEventTermination extends TaskEvent implements DiagnosableEvent,
-    TaskAttemptEventTerminationCauseEvent {
+    TaskAttemptEventTerminationCauseEvent, RecoveryEvent {
 
   private final String diagnostics;
   private final TaskAttemptTerminationCause errorCause;
-  
+  private boolean fromRecovery;
+
   public TaskEventTermination(TezTaskID taskID, TaskAttemptTerminationCause errorCause, String diagnostics) {
     super(taskID, TaskEventType.T_TERMINATE);
     this.errorCause = errorCause;
@@ -37,6 +38,12 @@ public class TaskEventTermination extends TaskEvent implements DiagnosableEvent,
     }
   }
 
+  public TaskEventTermination(TezTaskID taskID, TaskAttemptTerminationCause errorCause, String diagnostics,
+      boolean fromRecovery) {
+    this(taskID, errorCause, diagnostics);
+    this.fromRecovery = fromRecovery;
+  }
+
   @Override
   public String getDiagnosticInfo() {
     return diagnostics;
@@ -47,4 +54,9 @@ public class TaskEventTermination extends TaskEvent implements DiagnosableEvent,
     return errorCause;
   }
 
+  @Override
+  public boolean isFromRecovery() {
+    return fromRecovery;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
index baec5f0..726e13e 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskEventType.java
@@ -38,7 +38,4 @@ public enum TaskEventType {
   T_ATTEMPT_SUCCEEDED,
   T_ATTEMPT_KILLED,
 
-  // Recovery event
-  T_RECOVER
-
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRecoverVertex.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRecoverVertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRecoverVertex.java
index 34e45fe..4203689 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRecoverVertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRecoverVertex.java
@@ -32,5 +32,4 @@ public class VertexEventRecoverVertex extends VertexEvent {
   public VertexState getDesiredState() {
     return desiredState;
   }
-
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
index 69195db..211202d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventRouteEvent.java
@@ -27,25 +27,13 @@ public class VertexEventRouteEvent extends VertexEvent {
   
   final List<TezEvent> events;
 
-  final boolean recovered;
-
   public VertexEventRouteEvent(TezVertexID vertexId, List<TezEvent> events) {
-    this(vertexId, events, false);
-  }
-
-  public VertexEventRouteEvent(TezVertexID vertexId, List<TezEvent> events,
-      boolean recovered) {
     super(vertexId, VertexEventType.V_ROUTE_EVENT);
     this.events = events;
-    this.recovered = recovered;
   }
 
   public List<TezEvent> getEvents() {
     return events;
   }
 
-  public boolean isRecovered() {
-    return recovered;
-  }
-
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceVertexRecovered.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceVertexRecovered.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceVertexRecovered.java
deleted file mode 100644
index e3b9334..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventSourceVertexRecovered.java
+++ /dev/null
@@ -1,62 +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.dag.event;
-
-import org.apache.tez.dag.app.dag.VertexState;
-import org.apache.tez.dag.records.TezTaskAttemptID;
-import org.apache.tez.dag.records.TezVertexID;
-
-import java.util.List;
-
-public class VertexEventSourceVertexRecovered extends VertexEvent {
-
-  VertexState sourceVertexState;
-  TezVertexID sourceVertexID;
-  List<TezTaskAttemptID> completedTaskAttempts;
-  int sourceDistanceFromRoot;
-
-  public VertexEventSourceVertexRecovered(TezVertexID vertexID,
-      TezVertexID sourceVertexID,
-      VertexState sourceVertexState,
-      List<TezTaskAttemptID> completedTaskAttempts,
-      int sourceDistanceFromRoot) {
-    super(vertexID, VertexEventType.V_SOURCE_VERTEX_RECOVERED);
-    this.sourceVertexState = sourceVertexState;
-    this.sourceVertexID = sourceVertexID;
-    this.completedTaskAttempts = completedTaskAttempts;
-    this.sourceDistanceFromRoot = sourceDistanceFromRoot;
-  }
-
-  public VertexState getSourceVertexState() {
-    return sourceVertexState;
-  }
-
-  public TezVertexID getSourceVertexID() {
-    return sourceVertexID;
-  }
-
-  public List<TezTaskAttemptID> getCompletedTaskAttempts() {
-    return completedTaskAttempts;
-  }
-
-  public int getSourceDistanceFromRoot() {
-    return sourceDistanceFromRoot;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
index 6ea945b..15be94d 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/VertexEventType.java
@@ -57,9 +57,6 @@ public enum VertexEventType {
   
   // Producer: Vertex
   V_READY_TO_INIT,
-
-  // Recover Event, Producer:Vertex
-  V_SOURCE_VERTEX_RECOVERED,
   
   // Producer: Edge
   V_NULL_EDGE_INITIALIZED,

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
index 4dfba84..f395e62 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/DAGImpl.java
@@ -31,7 +31,6 @@ import java.util.Iterator;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Map.Entry;
 import java.util.Set;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.locks.Condition;
@@ -82,7 +81,9 @@ import org.apache.tez.dag.api.records.DAGProtos.PlanKeyValuePair;
 import org.apache.tez.dag.api.records.DAGProtos.PlanVertexGroupInfo;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
 import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.RecoveryParser.VertexRecoveryData;
 import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
+import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.DAGReport;
@@ -112,11 +113,11 @@ import org.apache.tez.dag.app.dag.event.VertexEventTermination;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
 import org.apache.tez.common.security.ACLManager;
 import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.events.DAGCommitStartedEvent;
 import org.apache.tez.dag.history.events.DAGFinishedEvent;
 import org.apache.tez.dag.history.events.DAGInitializedEvent;
 import org.apache.tez.dag.history.events.DAGStartedEvent;
+import org.apache.tez.dag.history.events.VertexFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
 import org.apache.tez.dag.records.TezDAGID;
@@ -204,10 +205,6 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
   private final List<String> diagnostics = new ArrayList<String>();
 
-  // Recovery related flags
-  boolean recoveryInitEventSeen = false;
-  boolean recoveryStartEventSeen = false;
-
   private TaskSpecificLaunchCmdOption taskSpecificLaunchCmdOption;
 
   private static final DagStateChangedCallback STATE_CHANGED_CALLBACK = new DagStateChangedCallback();
@@ -237,10 +234,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
           .addTransition(DAGState.NEW, DAGState.NEW,
               DAGEventType.DAG_DIAGNOSTIC_UPDATE,
               DIAGNOSTIC_UPDATE_TRANSITION)
+          // either recovered to FINISHED state or recovered to NEW to rerun the dag based on the recovery data
           .addTransition(DAGState.NEW,
-              EnumSet.of(DAGState.NEW, DAGState.INITED, DAGState.RUNNING,
-                  DAGState.SUCCEEDED, DAGState.FAILED, DAGState.KILLED,
-                  DAGState.ERROR, DAGState.TERMINATING),
+              EnumSet.of(DAGState.NEW, DAGState.SUCCEEDED,
+                  DAGState.FAILED, DAGState.KILLED,
+                  DAGState.ERROR),
               DAGEventType.DAG_RECOVER,
               new RecoverTransition())
           .addTransition(DAGState.NEW, DAGState.NEW,
@@ -448,11 +446,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
   Map<String, VertexGroupInfo> vertexGroups = Maps.newHashMap();
   Map<String, List<VertexGroupInfo>> vertexGroupInfo = Maps.newHashMap();
-  private DAGState recoveredState = DAGState.NEW;
-
-  @VisibleForTesting
-  boolean recoveryCommitInProgress = false;
-  Map<String, Boolean> recoveredGroupCommits = new HashMap<String, Boolean>();
+  private DAGRecoveryData recoveryData;
 
   static class VertexGroupInfo {
     String groupName;
@@ -637,59 +631,6 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   }
 
   @Override
-  public DAGState restoreFromEvent(HistoryEvent historyEvent) {
-    writeLock.lock();
-    try {
-      switch (historyEvent.getEventType()) {
-        case DAG_INITIALIZED:
-          recoveredState = initializeDAG((DAGInitializedEvent) historyEvent);
-          recoveryInitEventSeen = true;
-          return recoveredState;
-        case DAG_STARTED:
-          if (!recoveryInitEventSeen) {
-            throw new RuntimeException("Started Event seen but"
-                + " no Init Event was encountered earlier");
-          }
-          recoveryStartEventSeen = true;
-          this.startTime = ((DAGStartedEvent) historyEvent).getStartTime();
-          recoveredState = DAGState.RUNNING;
-          return recoveredState;
-        case DAG_COMMIT_STARTED:
-          recoveryCommitInProgress = true;
-          return recoveredState;
-        case VERTEX_GROUP_COMMIT_STARTED:
-          VertexGroupCommitStartedEvent vertexGroupCommitStartedEvent =
-              (VertexGroupCommitStartedEvent) historyEvent;
-          recoveredGroupCommits.put(
-              vertexGroupCommitStartedEvent.getVertexGroupName(), false);
-          return recoveredState;
-        case VERTEX_GROUP_COMMIT_FINISHED:
-          VertexGroupCommitFinishedEvent vertexGroupCommitFinishedEvent =
-              (VertexGroupCommitFinishedEvent) historyEvent;
-          recoveredGroupCommits.put(
-              vertexGroupCommitFinishedEvent.getVertexGroupName(), true);
-          return recoveredState;
-        case DAG_KILL_REQUEST:
-          trySetTerminationCause(DAGTerminationCause.DAG_KILL);
-          this.recoveredState = DAGState.KILLED;
-          return recoveredState;
-        case DAG_FINISHED:
-          recoveryCommitInProgress = false;
-          DAGFinishedEvent finishedEvent = (DAGFinishedEvent) historyEvent;
-          setFinishTime(finishedEvent.getFinishTime());
-          recoveredState = finishedEvent.getState();
-          this.fullCounters = finishedEvent.getTezCounters();
-          return recoveredState;
-        default:
-          throw new RuntimeException("Unexpected event received for restoring"
-              + " state, eventType=" + historyEvent.getEventType());
-      }
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
-  @Override
   public ACLManager getACLManager() {
     return this.aclManager;
   }
@@ -1241,39 +1182,50 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     return taskStats;
   }
 
-  void logJobHistoryFinishedEvent() throws IOException {
-    this.setFinishTime();
-    Map<String, Integer> taskStats = constructTaskStats(getDAGProgress());
-    DAGFinishedEvent finishEvt = new DAGFinishedEvent(dagId, startTime,
-        finishTime, DAGState.SUCCEEDED, "", getAllCounters(),
-        this.userName, this.dagName, taskStats, this.appContext.getApplicationAttemptId());
-    this.appContext.getHistoryHandler().handleCriticalEvent(
-        new DAGHistoryEvent(dagId, finishEvt));
-  }
-
   void logJobHistoryInitedEvent() {
-    DAGInitializedEvent initEvt = new DAGInitializedEvent(this.dagId,
-        this.initTime, this.userName, this.dagName, this.getVertexNameIDMapping());
-    this.appContext.getHistoryHandler().handle(
-        new DAGHistoryEvent(dagId, initEvt));
+    if (recoveryData == null
+        || recoveryData.getDAGInitializedEvent() == null) {
+      DAGInitializedEvent initEvt = new DAGInitializedEvent(this.dagId,
+          clock.getTime(), this.userName, this.dagName, this.getVertexNameIDMapping());
+      this.appContext.getHistoryHandler().handle(
+          new DAGHistoryEvent(dagId, initEvt));
+    }
   }
 
   void logJobHistoryStartedEvent() {
-    DAGStartedEvent startEvt = new DAGStartedEvent(this.dagId,
-        this.startTime, this.userName, this.dagName);
-    this.appContext.getHistoryHandler().handle(
-        new DAGHistoryEvent(dagId, startEvt));
+    if (recoveryData == null
+        || recoveryData.getDAGStartedEvent() == null) {
+      DAGStartedEvent startEvt = new DAGStartedEvent(this.dagId,
+          clock.getTime(), this.userName, this.dagName);
+      this.appContext.getHistoryHandler().handle(
+          new DAGHistoryEvent(dagId, startEvt));
+    }
+  }
+
+  void logJobHistoryFinishedEvent() throws IOException {
+    if (recoveryData == null
+        || recoveryData.getDAGFinishedEvent() == null) {
+      Map<String, Integer> taskStats = constructTaskStats(getDAGProgress());
+      DAGFinishedEvent finishEvt = new DAGFinishedEvent(dagId, clock.getTime(),
+          finishTime, DAGState.SUCCEEDED, "", getAllCounters(),
+          this.userName, this.dagName, taskStats, this.appContext.getApplicationAttemptId());
+      this.appContext.getHistoryHandler().handleCriticalEvent(
+          new DAGHistoryEvent(dagId, finishEvt));
+    }
   }
 
   void logJobHistoryUnsuccesfulEvent(DAGState state) throws IOException {
-    Map<String, Integer> taskStats = constructTaskStats(getDAGProgress());
-    DAGFinishedEvent finishEvt = new DAGFinishedEvent(dagId, startTime,
-        clock.getTime(), state,
-        StringUtils.join(getDiagnostics(), LINE_SEPARATOR),
-        getAllCounters(), this.userName, this.dagName, taskStats,
-        this.appContext.getApplicationAttemptId());
-    this.appContext.getHistoryHandler().handleCriticalEvent(
-        new DAGHistoryEvent(dagId, finishEvt));
+    if (recoveryData == null
+        || recoveryData.getDAGFinishedEvent() == null) {
+      Map<String, Integer> taskStats = constructTaskStats(getDAGProgress());
+      DAGFinishedEvent finishEvt = new DAGFinishedEvent(dagId, 0L,
+          clock.getTime(), state,
+          StringUtils.join(getDiagnostics(), LINE_SEPARATOR),
+          getAllCounters(), this.userName, this.dagName, taskStats,
+          this.appContext.getApplicationAttemptId());
+      this.appContext.getHistoryHandler().handleCriticalEvent(
+          new DAGHistoryEvent(dagId, finishEvt));
+    }
   }
 
   // triggered by vertex_complete
@@ -1474,17 +1426,8 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     }
   }
 
-  public DAGState initializeDAG() {
-    return initializeDAG(null);
-  }
-
-  DAGState initializeDAG(DAGInitializedEvent event) {
-    if (event != null) {
-      initTime = event.getInitTime();
-    } else {
-      initTime = clock.getTime();
-    }
 
+  DAGState initializeDAG() {
     commitAllOutputsOnSuccess = dagConf.getBoolean(
         TezConfiguration.TEZ_AM_COMMIT_ALL_OUTPUTS_ON_DAG_SUCCESS,
         TezConfiguration.TEZ_AM_COMMIT_ALL_OUTPUTS_ON_DAG_SUCCESS_DEFAULT);
@@ -1494,9 +1437,6 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     if (numVertices == 0) {
       addDiagnostic("No vertices for dag");
       trySetTerminationCause(DAGTerminationCause.ZERO_VERTICES);
-      if (event != null) {
-        return DAGState.FAILED;
-      }
       return finished(DAGState.FAILED);
     }
 
@@ -1668,139 +1608,68 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     vertex.setOutputVertices(outVertices);
   }
 
+  /**
+   * 2 cases of recovery:
+   * <ul>
+   * <li>
+   * 1. For the completed dag, recover the dag to the desired state and also its vertices,
+   *    but not task & task attempt. This recovery is sync call (after this Transition, 
+   *    DAG & vertices are all recovered to the desired state)
+   * </li>
+   * <li>
+   * 2. For the non-completed dag, recover the dag as normal dag execution. The only difference
+   *    is setting the recoveryData before sending DAG_INIT event so that some steps in the execution
+   *    will be skipped based on the recoveryData
+   * </li>
+   * </ul>
+   */
   private static class RecoverTransition
       implements MultipleArcTransition<DAGImpl, DAGEvent, DAGState> {
 
     @Override
     public DAGState transition(DAGImpl dag, DAGEvent dagEvent) {
-      DAGEventRecoverEvent recoverEvent = (DAGEventRecoverEvent) dagEvent;
+      DAGEventRecoverEvent recoverEvent = (DAGEventRecoverEvent)dagEvent;
+      // With desired state, represents the case that DAG is completed
       if (recoverEvent.hasDesiredState()) {
-        // DAG completed or final end state known
-        dag.recoveredState = recoverEvent.getDesiredState();
-      }
-      if (recoverEvent.getAdditionalUrlsForClasspath() != null) {
-        LOG.info("Added additional resources : [" + recoverEvent.getAdditionalUrlsForClasspath()
-            + "] to classpath");
-        RelocalizationUtils.addUrlsToClassPath(recoverEvent.getAdditionalUrlsForClasspath());
-      }
-
-      switch (dag.recoveredState) {
-        case NEW:
-          // send DAG an Init and start events
-          dag.eventHandler.handle(new DAGEvent(dag.getID(), DAGEventType.DAG_INIT));
-          dag.eventHandler.handle(new DAGEventStartDag(dag.getID(), null));
-          return DAGState.NEW;
-        case INITED:
-          // DAG inited but not started
-          // This implies vertices need to be sent init event
-          // Root vertices need to be sent start event
-          // The vertices may already have been sent these events but the
-          // DAG start may not have been persisted
-          for (Vertex v : dag.vertices.values()) {
-            if (v.getInputVerticesCount() == 0) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Sending Running Recovery event to root vertex "
-                    + v.getLogIdentifier());
-              }
-              dag.eventHandler.handle(new VertexEventRecoverVertex(v.getVertexId(),
-                  VertexState.RUNNING));
-            }
-          }
-          return DAGState.RUNNING;
-        case RUNNING:
-          // if commit is in progress, DAG should fail as commits are not
-          // recoverable
-          boolean groupCommitInProgress = false;
-          if (!dag.recoveredGroupCommits.isEmpty()) {
-            for (Entry<String, Boolean> entry : dag.recoveredGroupCommits.entrySet()) {
-              if (!entry.getValue().booleanValue()) {
-                LOG.info("Found a pending Vertex Group commit"
-                    + ", vertexGroup=" + entry.getKey());
-                groupCommitInProgress = true;
-                break;
-              }
-            }
-          }
-
-          if (groupCommitInProgress || dag.recoveryCommitInProgress) {
-            // Fail the DAG as we have not seen a commit completion
-            dag.trySetTerminationCause(DAGTerminationCause.COMMIT_FAILURE);
-            dag.setFinishTime();
-            // Recover all other data for all vertices
-            // send recover event to all vertices with a final end state
-            for (Vertex v : dag.vertices.values()) {
-              VertexState desiredState = VertexState.SUCCEEDED;
-              if (dag.recoveredState.equals(DAGState.KILLED)) {
-                desiredState = VertexState.KILLED;
-              } else if (EnumSet.of(DAGState.ERROR, DAGState.FAILED).contains(
-                  dag.recoveredState)) {
-                desiredState = VertexState.FAILED;
-              }
-              dag.eventHandler.handle(new VertexEventRecoverVertex(v.getVertexId(),
-                  desiredState));
-            }
-            DAGState endState = DAGState.FAILED;
-            try {
-              dag.logJobHistoryUnsuccesfulEvent(endState);
-            } catch (IOException e) {
-              LOG.warn("Failed to persist recovery event for DAG completion"
-                  + ", dagId=" + dag.dagId
-                  + ", finalState=" + endState);
-            }
-            dag.eventHandler.handle(new DAGAppMasterEventDAGFinished(dag.getID(),
-                endState));
-            return endState;
-          }
-
-          for (Vertex v : dag.vertices.values()) {
-            if (v.getInputVerticesCount() == 0) {
-              if (LOG.isDebugEnabled()) {
-                LOG.debug("Sending Running Recovery event to root vertex "
-                    + v.getLogIdentifier());
-              }
-              dag.eventHandler.handle(new VertexEventRecoverVertex(v.getVertexId(),
-                  VertexState.RUNNING));
-            }
-          }
-          return DAGState.RUNNING;
+        VertexState vertexDesiredState = null;
+        switch (recoverEvent.getDesiredState()) {
         case SUCCEEDED:
-        case ERROR:
+          vertexDesiredState = VertexState.SUCCEEDED;
+          break;
         case FAILED:
+          vertexDesiredState = VertexState.FAILED;
+          break;
         case KILLED:
-          // Completed
-          
-          // Recover all other data for all vertices
-          // send recover event to all vertices with a final end state
-          for (Vertex v : dag.vertices.values()) {
-            VertexState desiredState = VertexState.SUCCEEDED;
-            if (dag.recoveredState.equals(DAGState.KILLED)) {
-              desiredState = VertexState.KILLED;
-            } else if (EnumSet.of(DAGState.ERROR, DAGState.FAILED).contains(
-                dag.recoveredState)) {
-              desiredState = VertexState.FAILED;
-            }
-            dag.eventHandler.handle(new VertexEventRecoverVertex(v.getVertexId(),
-                desiredState));
-          }
-
-          // Let us inform AM of completion
-          dag.eventHandler.handle(new DAGAppMasterEventDAGFinished(dag.getID(),
-              dag.recoveredState));
-
-          LOG.info("Recovered DAG: " + dag.getID() + " finished with state: "
-              + dag.recoveredState);
-          return dag.recoveredState;
+          vertexDesiredState = VertexState.KILLED;
+          break;
+        case ERROR:
+          vertexDesiredState = VertexState.ERROR;
+          break;
         default:
-          // Error state
-          LOG.warn("Trying to recover DAG, failed to recover"
-              + " from non-handled state" + dag.recoveredState);
-          // Tell AM ERROR so that it can shutdown
-          dag.eventHandler.handle(new DAGAppMasterEventDAGFinished(dag.getID(),
-              DAGState.ERROR));
-          return DAGState.FAILED;
+          String msg = "Invalid desired state of DAG"
+              + ", dagName=" + dag.getName()
+              + ", state=" + recoverEvent.getDesiredState();
+          LOG.warn(msg);
+          dag.addDiagnostic(msg);
+          return dag.finished(DAGState.ERROR);
+        }
+        // Initialize dag synchronously to generate the vertices and recover its vertices to the desired state.
+        dag.initializeDAG();
+        for (Vertex v : dag.vertexMap.values()) {
+          dag.eventHandler.handle(new VertexEventRecoverVertex(v.getVertexId(), vertexDesiredState));
+        }
+        dag.addDiagnostic("DAG is recovered to finished state:" + recoverEvent.getDesiredState()
+            + ", but will only recover partial data due to incomplete recovery data");
+        return dag.finished(recoverEvent.getDesiredState());
       }
-    }
 
+      // for the cases that DAG is not completed, recover it as normal dag execution.
+      dag.recoveryData = recoverEvent.getRecoveredDagData();
+      dag.appContext.setDAGRecoveryData(dag.recoveryData);
+      dag.getEventHandler().handle(new DAGEvent(dag.getID(), DAGEventType.DAG_INIT));
+      dag.getEventHandler().handle(new DAGEventStartDag(dag.getID(), dag.recoveryData.additionalUrlsForClasspath));
+      return DAGState.NEW;
+    }
   }
 
   private static class InitTransition
@@ -1818,6 +1687,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
       // TODO Metrics
       //dag.metrics.submittedJob(dag);
       //dag.metrics.preparingJob(dag);
+      if (dag.recoveryData != null && dag.recoveryData.getDAGInitializedEvent() != null) {
+        dag.initTime = dag.recoveryData.getDAGInitializedEvent().getInitTime();
+      } else {
+        dag.initTime = dag.clock.getTime();
+      }
       dag.startDAGCpuTime = dag.appContext.getCumulativeCPUTime();
       dag.startDAGGCTime = dag.appContext.getCumulativeGCTime();
 
@@ -1845,9 +1719,12 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
      */
     @Override
     public void transition(DAGImpl dag, DAGEvent event) {
+      if (dag.recoveryData != null && dag.recoveryData.getDAGStartedEvent() != null) {
+        dag.startTime = dag.recoveryData.getDAGStartedEvent().getStartTime();
+      } else {
+        dag.startTime = dag.clock.getTime();
+      }
       DAGEventStartDag startEvent = (DAGEventStartDag) event;
-      dag.startTime = dag.clock.getTime();
-      dag.logJobHistoryStartedEvent();
       List<URL> additionalUrlsForClasspath = startEvent.getAdditionalUrlsForClasspath();
       if (additionalUrlsForClasspath != null) {
         LOG.info("Added additional resources : [" + additionalUrlsForClasspath  + "] to classpath");
@@ -1858,6 +1735,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
       // Start all vertices with no incoming edges when job starts
       dag.initializeVerticesAndStart();
+      dag.logJobHistoryStartedEvent();
     }
   }
 
@@ -2032,6 +1910,14 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
 
   }
 
+  private Collection<TezVertexID> getVertexIds(Collection<String> vertexNames) {
+    List<TezVertexID> vertexIds = new ArrayList<TezVertexID>(vertexNames.size());
+    for (String name : vertexNames) {
+      vertexIds.add(getVertexNameIDMapping().get(name));
+    }
+    return vertexIds;
+  }
+
   private static class VertexReRunningTransition implements
     MultipleArcTransition<DAGImpl, DAGEvent, DAGState> {
 
@@ -2078,17 +1964,30 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
           }
         }
         for (VertexGroupInfo groupInfo : commitList) {
-          if (recoveredGroupCommits.containsKey(groupInfo.groupName)) {
+          if (recoveryData != null && recoveryData.isVertexGroupCommitted(groupInfo.groupName)) {
             LOG.info("VertexGroup was already committed as per recovery"
                 + " data, groupName=" + groupInfo.groupName);
+            for (String vertexName : groupInfo.groupMembers) {
+              VertexRecoveryData vertexRecoveryData =
+                  recoveryData.getVertexRecoveryData(getVertex(vertexName).getVertexId());
+              Preconditions.checkArgument(vertexRecoveryData != null,"Vertex Group has been committed"
+                  + ", but no VertexRecoveryData found for its vertex " + vertexName);
+              VertexFinishedEvent vertexFinishedEvent = vertexRecoveryData.getVertexFinishedEvent();
+              Preconditions.checkArgument(vertexFinishedEvent!= null,"Vertex Group has been committed"
+                  + ", but no VertexFinishedEvent found in its vertex " + vertexName);
+              Preconditions.checkArgument(vertexFinishedEvent.getState() == VertexState.SUCCEEDED,
+                  "Vertex Group has been committed, but unexpected vertex state of its vertex "
+                  + vertexName + ", vertexstate=" + vertexFinishedEvent.getState());
+            }
             continue;
           }
           groupInfo.commitStarted = true;
           final Vertex v = getVertex(groupInfo.groupMembers.iterator().next());
           try {
+            Collection<TezVertexID> vertexIds = getVertexIds(groupInfo.groupMembers);
             appContext.getHistoryHandler().handleCriticalEvent(new DAGHistoryEvent(getID(),
                 new VertexGroupCommitStartedEvent(dagId, groupInfo.groupName,
-                    clock.getTime())));
+                    vertexIds, clock.getTime())));
           } catch (IOException e) {
             LOG.error("Failed to send commit recovery event to handler", e);
             recoveryFailed = true;
@@ -2269,9 +2168,10 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
         if (vertexGroup.isCommitted()) {
           if (!commitAllOutputsOnSuccess) {
             try {
+              Collection<TezVertexID> vertexIds = getVertexIds(vertexGroup.groupMembers);
               appContext.getHistoryHandler().handleCriticalEvent(new DAGHistoryEvent(getID(),
                   new VertexGroupCommitFinishedEvent(getID(), commitCompletedEvent.getOutputKey().getEntityName(),
-                      clock.getTime())));
+                      vertexIds, clock.getTime())));
             } catch (IOException e) {
               String diag = "Failed to send commit recovery event to handler, " + ExceptionUtils.getStackTrace(e);
               addDiagnostic(diag);

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
index bfd1634..957abcf 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskAttemptImpl.java
@@ -61,6 +61,7 @@ import org.apache.tez.dag.api.oldrecords.TaskAttemptReport;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.RecoveryParser.TaskAttemptRecoveryData;
 import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.Task;
@@ -72,9 +73,13 @@ import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate;
 import org.apache.tez.dag.app.dag.event.DAGEventDiagnosticsUpdate;
 import org.apache.tez.dag.app.dag.event.DAGEventType;
 import org.apache.tez.dag.app.dag.event.DiagnosableEvent;
+import org.apache.tez.dag.app.dag.event.RecoveryEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventContainerTerminated;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventKillRequest;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventTezEventUpdate;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventTerminationCauseEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventSchedule;
@@ -89,7 +94,6 @@ import org.apache.tez.dag.app.rm.AMSchedulerEventTAEnded;
 import org.apache.tez.dag.app.rm.AMSchedulerEventTALaunchRequest;
 import org.apache.tez.dag.app.rm.container.AMContainer;
 import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
 import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
@@ -109,6 +113,7 @@ import org.apache.tez.runtime.api.impl.TezEvent;
 import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
@@ -166,6 +171,7 @@ public class TaskAttemptImpl implements TaskAttempt,
   private final Lock writeLock;
   protected final AppContext appContext;
   private final TaskHeartbeatHandler taskHeartbeatHandler;
+  private final TaskAttemptRecoveryData recoveryData;
   private long launchTime = 0;
   private long finishTime = 0;
   private String trackerName;
@@ -191,10 +197,12 @@ public class TaskAttemptImpl implements TaskAttempt,
   private DAGCounter localityCounter;
   
   org.apache.tez.runtime.api.impl.TaskStatistics statistics;
-  
+
   long lastNotifyProgressTimestamp = 0;
   private final long hungIntervalMax;
 
+  private List<TezEvent> taGeneratedEvents = Lists.newArrayList();
+
   // Used to store locality information when
   Set<String> taskHosts = new HashSet<String>();
   Set<String> taskRacks = new HashSet<String>();
@@ -240,20 +248,29 @@ public class TaskAttemptImpl implements TaskAttempt,
             (TaskAttemptStateInternal.NEW)
 
       .addTransition(TaskAttemptStateInternal.NEW,
-          EnumSet.of(TaskAttemptStateInternal.START_WAIT, TaskAttemptStateInternal.FAILED),
+          EnumSet.of(TaskAttemptStateInternal.NEW, TaskAttemptStateInternal.START_WAIT, TaskAttemptStateInternal.FAILED),
           TaskAttemptEventType.TA_SCHEDULE, new ScheduleTaskattemptTransition())
+       // NEW -> FAILED due to TA_FAILED happens in recovery 
+       // (No TaskAttemptStartedEvent, but with TaskAttemptFinishedEvent(FAILED)
+      .addTransition(TaskAttemptStateInternal.NEW,
+              TaskAttemptStateInternal.FAILED,
+          TaskAttemptEventType.TA_FAILED, new TerminateTransition(FAILED_HELPER))
       .addTransition(TaskAttemptStateInternal.NEW,
           TaskAttemptStateInternal.KILLED,
           TaskAttemptEventType.TA_KILL_REQUEST,
           new TerminateTransition(KILLED_HELPER))
-
+      // NEW -> KILLED due to TA_KILLED happens in recovery
+      // (No TaskAttemptStartedEvent, but with TaskAttemptFinishedEvent(KILLED)    
       .addTransition(TaskAttemptStateInternal.NEW,
-          EnumSet.of(TaskAttemptStateInternal.NEW,
-              TaskAttemptStateInternal.RUNNING,
-              TaskAttemptStateInternal.KILLED,
-              TaskAttemptStateInternal.FAILED,
-              TaskAttemptStateInternal.SUCCEEDED),
-          TaskAttemptEventType.TA_RECOVER, new RecoverTransition())
+          TaskAttemptStateInternal.KILLED,
+          TaskAttemptEventType.TA_KILLED,
+          new TerminateTransition(KILLED_HELPER))
+      // NEW -> SUCCEEDED due to TA_DONE happens in recovery
+      // (with TaskAttemptStartedEvent and with TaskAttemptFinishedEvent(SUCCEEDED)    
+      .addTransition(TaskAttemptStateInternal.NEW,
+          TaskAttemptStateInternal.SUCCEEDED,
+          TaskAttemptEventType.TA_DONE,
+          new SucceededTransition())
 
       .addTransition(TaskAttemptStateInternal.START_WAIT,
           TaskAttemptStateInternal.RUNNING,
@@ -328,6 +345,11 @@ public class TaskAttemptImpl implements TaskAttempt,
               TaskAttemptStateInternal.RUNNING),
           TaskAttemptEventType.TA_OUTPUT_FAILED,
           new OutputReportedFailedTransition())
+       // for recovery, needs to log the TA generated events in TaskAttemptFinishedEvent    
+      .addTransition(TaskAttemptStateInternal.RUNNING,
+          TaskAttemptStateInternal.RUNNING,
+          TaskAttemptEventType.TA_TEZ_EVENT_UPDATE,
+          new TezEventUpdaterTransition())
 
       .addTransition(TaskAttemptStateInternal.KILL_IN_PROGRESS,
           TaskAttemptStateInternal.KILLED,
@@ -434,9 +456,6 @@ public class TaskAttemptImpl implements TaskAttempt,
 
         .installTopology();
 
-  private TaskAttemptState recoveredState = TaskAttemptState.NEW;
-  private boolean recoveryStartEventSeen = false;
-
   @SuppressWarnings("rawtypes")
   public TaskAttemptImpl(TezTaskID taskId, int attemptNumber, EventHandler eventHandler,
       TaskCommunicatorManagerInterface taskCommunicatorManagerInterface, Configuration conf, Clock clock,
@@ -493,6 +512,8 @@ public class TaskAttemptImpl implements TaskAttempt,
         TezConfiguration.TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS, 
         TezConfiguration.TEZ_TASK_PROGRESS_STUCK_INTERVAL_MS_DEFAULT);
 
+    this.recoveryData = appContext.getDAGRecoveryData() == null ?
+        null : appContext.getDAGRecoveryData().getTaskAttemptRecoveryData(attemptId);
   }
 
   @Override
@@ -521,14 +542,6 @@ public class TaskAttemptImpl implements TaskAttempt,
 
   TaskSpec createRemoteTaskSpec() throws AMUserCodeException {
     TaskSpec baseTaskSpec = task.getBaseTaskSpec();
-    if (baseTaskSpec == null) {
-      // since recovery does not follow normal transitions, TaskEventScheduleTask
-      // is not being honored by the recovery code path. Using this to workaround 
-      // until recovery is fixed. Calling the non-locking internal method of the vertex
-      // to get the taskSpec directly. Since everything happens on the central dispatcher 
-      // during recovery this is deadlock free for now. TEZ-1019 should remove the need for this.
-      baseTaskSpec = ((VertexImpl) vertex).createRemoteTaskSpec(getID().getTaskID().getId());
-    }
     return new TaskSpec(getID(),
         baseTaskSpec.getDAGName(), baseTaskSpec.getVertexName(),
         baseTaskSpec.getVertexParallelism(), baseTaskSpec.getProcessorDescriptor(),
@@ -839,52 +852,6 @@ public class TaskAttemptImpl implements TaskAttempt,
     }
   }
 
-  @Override
-  public TaskAttemptState restoreFromEvent(HistoryEvent historyEvent) {
-    writeLock.lock();
-    try {
-      switch (historyEvent.getEventType()) {
-        case TASK_ATTEMPT_STARTED:
-        {
-          TaskAttemptStartedEvent tEvent = (TaskAttemptStartedEvent) historyEvent;
-          this.launchTime = tEvent.getStartTime();
-          recoveryStartEventSeen = true;
-          recoveredState = TaskAttemptState.RUNNING;
-          this.containerId = tEvent.getContainerId();
-          sendEvent(createDAGCounterUpdateEventTALaunched(this));
-          return recoveredState;
-        }
-        case TASK_ATTEMPT_FINISHED:
-        {
-          TaskAttemptFinishedEvent tEvent = (TaskAttemptFinishedEvent) historyEvent;
-          this.creationTime = tEvent.getCreationTime();
-          this.allocationTime = tEvent.getAllocationTime();
-          this.launchTime = tEvent.getStartTime();
-          this.finishTime = tEvent.getFinishTime();
-          this.creationCausalTA = tEvent.getCreationCausalTA();
-          this.reportedStatus.counters = tEvent.getCounters();
-          this.reportedStatus.progress = 1f;
-          this.reportedStatus.state = tEvent.getState();
-          this.terminationCause = tEvent.getTaskAttemptError() != null ? tEvent.getTaskAttemptError()
-              : TaskAttemptTerminationCause.UNKNOWN_ERROR;
-          this.diagnostics.add(tEvent.getDiagnostics());
-          this.recoveredState = tEvent.getState();
-          if (tEvent.getDataEvents() != null) {
-            this.lastDataEvents.addAll(tEvent.getDataEvents());
-          }
-          sendEvent(createDAGCounterUpdateEventTAFinished(this, tEvent.getState()));
-          return recoveredState;
-        }
-        default:
-          throw new RuntimeException("Unexpected event received for restoring"
-              + " state, eventType=" + historyEvent.getEventType());
-  
-      }
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
   @SuppressWarnings("unchecked")
   private void sendEvent(Event<?> event) {
     this.eventHandler.handle(event);
@@ -1055,6 +1022,7 @@ public class TaskAttemptImpl implements TaskAttempt,
   }
 
   protected void logJobHistoryAttemptStarted() {
+    Preconditions.checkArgument(recoveryData == null);
     final String containerIdStr = containerId.toString();
     String inProgressLogsUrl = nodeHttpAddress
        + "/" + "node/containerlogs"
@@ -1081,13 +1049,16 @@ public class TaskAttemptImpl implements TaskAttempt,
   }
 
   protected void logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal state) {
-    //Log finished events only if an attempt started.
+    Preconditions.checkArgument(recoveryData == null
+        || recoveryData.getTaskAttemptFinishedEvent() == null,
+        "log TaskAttemptFinishedEvent again in recovery when there's already another TaskAtttemptFinishedEvent");
     if (getLaunchTime() == 0) return;
 
     TaskAttemptFinishedEvent finishEvt = new TaskAttemptFinishedEvent(
         attemptId, getVertex().getName(), getLaunchTime(),
         getFinishTime(), TaskAttemptState.SUCCEEDED, null,
-        "", getCounters(), lastDataEvents, creationTime, creationCausalTA, allocationTime);
+        "", getCounters(), lastDataEvents, taGeneratedEvents,
+        creationTime, creationCausalTA, allocationTime);
     // FIXME how do we store information regd completion events
     this.appContext.getHistoryHandler().handle(
         new DAGHistoryEvent(getDAGID(), finishEvt));
@@ -1095,6 +1066,9 @@ public class TaskAttemptImpl implements TaskAttempt,
 
   protected void logJobHistoryAttemptUnsuccesfulCompletion(
       TaskAttemptState state) {
+    Preconditions.checkArgument(recoveryData == null
+        || recoveryData.getTaskAttemptFinishedEvent() == null,
+        "log TaskAttemptFinishedEvent again in recovery when there's already another TaskAtttemptFinishedEvent");
     long finishTime = getFinishTime();
     if (finishTime <= 0) {
       finishTime = clock.getTime(); // comes here in case it was terminated before launch
@@ -1104,8 +1078,8 @@ public class TaskAttemptImpl implements TaskAttempt,
         finishTime, state,
         terminationCause,
         StringUtils.join(
-            getDiagnostics(), LINE_SEPARATOR), getCounters(), lastDataEvents, 
-        creationTime, creationCausalTA, allocationTime);
+            getDiagnostics(), LINE_SEPARATOR), getCounters(), lastDataEvents,
+        taGeneratedEvents, creationTime, creationCausalTA, allocationTime);
     // FIXME how do we store information regd completion events
     this.appContext.getHistoryHandler().handle(
         new DAGHistoryEvent(getDAGID(), finishEvt));
@@ -1116,12 +1090,69 @@ public class TaskAttemptImpl implements TaskAttempt,
   //////////////////////////////////////////////////////////////////////////////
 
   protected static class ScheduleTaskattemptTransition implements
-      MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
+    MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
 
     @Override
     public TaskAttemptStateInternal transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
-      TaskAttemptEventSchedule scheduleEvent = (TaskAttemptEventSchedule) event;
+      if (ta.recoveryData != null) {
+        TaskAttemptStartedEvent taStartedEvent =
+            ta.recoveryData.getTaskAttemptStartedEvent();
+        if (taStartedEvent != null) {
+          ta.launchTime = taStartedEvent.getStartTime();
+          TaskAttemptFinishedEvent taFinishedEvent =
+              ta.recoveryData.getTaskAttemptFinishedEvent();
+          if (taFinishedEvent == null) {
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("Only TaskAttemptStartedEvent but no TaskAttemptFinishedEvent, "
+                  + "send out TaskAttemptEventAttemptKilled to move it to KILLED");
+            }
+            ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(), 
+                "Task Attempt killed in recovery due to can't recover the running task attempt",
+                TaskAttemptTerminationCause.TERMINATED_AT_RECOVERY, true));
+            return TaskAttemptStateInternal.NEW;
+          }
+        }
+        // No matter whether TaskAttemptStartedEvent is seen, send corresponding event to move 
+        // TA to the state of TaskAttemptFinishedEvent
+        TaskAttemptFinishedEvent taFinishedEvent =
+            ta.recoveryData.getTaskAttemptFinishedEvent();
+        Preconditions.checkArgument(taFinishedEvent != null, "Both of TaskAttemptStartedEvent and TaskFinishedEvent is null,"
+            + "taskAttemptId=" + ta.getID());
+        switch (taFinishedEvent.getState()) {
+          case FAILED:
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("TaskAttemptFinishedEvent is seen with state of FAILED"
+                  + ", send TA_FAILED to itself"
+                  + ", attemptId=" + ta.attemptId);
+            }
+            ta.sendEvent(new TaskAttemptEventAttemptFailed(ta.getID(), TaskAttemptEventType.TA_FAILED,
+                taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true));
+            break;
+          case KILLED:
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("TaskAttemptFinishedEvent is seen with state of KILLED"
+                  + ", send TA_KILLED to itself"
+                  + ", attemptId=" + ta.attemptId);
+            }
+            ta.sendEvent(new TaskAttemptEventAttemptKilled(ta.getID(),
+                taFinishedEvent.getDiagnostics(), taFinishedEvent.getTaskAttemptError(), true));
+            break;
+          case SUCCEEDED:
+            if (LOG.isDebugEnabled()) {
+              LOG.debug("TaskAttemptFinishedEvent is seen with state of SUCCEEDED"
+                  + ", send TA_DONE to itself"
+                  + ", attemptId=" + ta.attemptId);
+            }
+            ta.sendEvent(new TaskAttemptEvent(ta.getID(), TaskAttemptEventType.TA_DONE));
+            break;
+          default:
+            throw new TezUncheckedException("Invalid state in TaskAttemptFinishedEvent, state=" 
+                + taFinishedEvent.getState() + ", taId=" + ta.getID());
+        }
+        return TaskAttemptStateInternal.NEW;
+      }
 
+      TaskAttemptEventSchedule scheduleEvent = (TaskAttemptEventSchedule) event;
       ta.scheduledTime = ta.clock.getTime();
       // TODO Creating the remote task here may not be required in case of
       // recovery.
@@ -1212,7 +1243,14 @@ public class TaskAttemptImpl implements TaskAttempt,
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
       // This transition should not be invoked directly, if a scheduler event has already been sent out.
       // Sub-classes should be used if a scheduler request has been sent.
-      ta.setFinishTime();
+      if (ta.recoveryData == null ||
+          ta.recoveryData.getTaskAttemptFinishedEvent() == null) {
+        ta.setFinishTime();
+        ta.logJobHistoryAttemptUnsuccesfulCompletion(helper
+            .getTaskAttemptState());
+      } else {
+        ta.finishTime = ta.recoveryData.getTaskAttemptFinishedEvent().getFinishTime();
+      }
 
       if (event instanceof DiagnosableEvent) {
         ta.addDiagnosticInfo(((DiagnosableEvent) event).getDiagnosticInfo());
@@ -1225,11 +1263,16 @@ public class TaskAttemptImpl implements TaskAttempt,
             + ", requiredClass=TaskAttemptEventTerminationCauseEvent"
             + ", eventClass=" + event.getClass().getName());
       }
-
+      if (event instanceof RecoveryEvent) {
+        RecoveryEvent rEvent = (RecoveryEvent)event;
+        if (rEvent.isFromRecovery()) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Faked TerminateEvent from recovery, taskAttemptId=" + ta.getID());
+          }
+        }
+      }
       ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
           helper.getTaskAttemptState()));
-      ta.logJobHistoryAttemptUnsuccesfulCompletion(helper
-          .getTaskAttemptState());
       // Send out events to the Task - indicating TaskAttemptTermination(F/K)
       ta.sendEvent(new TaskEventTAUpdate(ta.attemptId, helper
           .getTaskEventType(), event));
@@ -1413,14 +1456,42 @@ public class TaskAttemptImpl implements TaskAttempt,
     }
   }
 
+  protected static class TezEventUpdaterTransition implements
+      SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
+
+    @Override
+    public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
+      TaskAttemptEventTezEventUpdate tezEventUpdate = (TaskAttemptEventTezEventUpdate)event;
+      ta.taGeneratedEvents.addAll(tezEventUpdate.getTezEvents());
+    }
+  }
+
   protected static class SucceededTransition implements
       SingleArcTransition<TaskAttemptImpl, TaskAttemptEvent> {
     @Override
     public void transition(TaskAttemptImpl ta, TaskAttemptEvent event) {
 
-      ta.setFinishTime();
-      // Send out history event.
-      ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
+      // If TaskAttempt is recovered to SUCCEEDED, send events generated by this TaskAttempt to vertex
+      // for its downstream consumers. For normal dag execution, the events are sent by TaskAttmeptListener
+      // for performance consideration.
+      if (ta.recoveryData != null && ta.recoveryData.isTaskAttemptSucceeded()) {
+        TaskAttemptFinishedEvent taFinishedEvent = ta.recoveryData
+            .getTaskAttemptFinishedEvent();
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("TaskAttempt is recovered to SUCCEEDED, attemptId=" + ta.attemptId);
+        }
+        ta.reportedStatus.counters = taFinishedEvent.getCounters();
+        List<TezEvent> tezEvents = taFinishedEvent.getTAGeneratedEvents();
+        if (tezEvents != null && !tezEvents.isEmpty()) {
+          ta.sendEvent(new VertexEventRouteEvent(ta.getVertexID(), tezEvents));
+        }
+        ta.finishTime = taFinishedEvent.getFinishTime();
+      } else {
+        ta.setFinishTime();
+        // Send out history event.
+        ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
+      }
+
       ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
           TaskAttemptState.SUCCEEDED));
 
@@ -1520,48 +1591,6 @@ public class TaskAttemptImpl implements TaskAttempt,
 
   }
 
-  protected static class RecoverTransition implements
-      MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
-
-    @Override
-    public TaskAttemptStateInternal transition(TaskAttemptImpl taskAttempt, TaskAttemptEvent taskAttemptEvent) {
-      TaskAttemptStateInternal endState = TaskAttemptStateInternal.FAILED;
-      switch(taskAttempt.recoveredState) {
-        case NEW:
-        case RUNNING:
-          // FIXME once running containers can be recovered, this
-          // should be handled differently
-          // TODO abort taskattempt
-          taskAttempt.sendEvent(new TaskEventTAUpdate(taskAttempt.attemptId,
-              TaskEventType.T_ATTEMPT_KILLED));
-          taskAttempt.sendEvent(createDAGCounterUpdateEventTAFinished(taskAttempt,
-              getExternalState(TaskAttemptStateInternal.KILLED)));
-          taskAttempt.logJobHistoryAttemptUnsuccesfulCompletion(TaskAttemptState.KILLED);
-          endState = TaskAttemptStateInternal.KILLED;
-          break;
-        case SUCCEEDED:
-          // Do not inform Task as it already knows about completed attempts
-          endState = TaskAttemptStateInternal.SUCCEEDED;
-          break;
-        case FAILED:
-          // Do not inform Task as it already knows about completed attempts
-          endState = TaskAttemptStateInternal.FAILED;
-          break;
-        case KILLED:
-          // Do not inform Task as it already knows about completed attempts
-          endState = TaskAttemptStateInternal.KILLED;
-          break;
-        default:
-          throw new RuntimeException("Failed to recover from non-handled state"
-              + ", taskAttemptId=" + taskAttempt.getID()
-              + ", state=" + taskAttempt.recoveredState);
-      }
-
-      return endState;
-    }
-
-  }
-
   protected static class TerminatedAfterSuccessTransition implements
       MultipleArcTransition<TaskAttemptImpl, TaskAttemptEvent, TaskAttemptStateInternal> {
     @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
index 2f304c8..55dd518 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/TaskImpl.java
@@ -57,6 +57,7 @@ import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.TaskCommunicatorManagerInterface;
+import org.apache.tez.dag.app.RecoveryParser.TaskRecoveryData;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.StateChangeNotifier;
 import org.apache.tez.dag.app.dag.Task;
@@ -68,12 +69,9 @@ import org.apache.tez.dag.app.dag.event.DAGEvent;
 import org.apache.tez.dag.app.dag.event.DAGEventDiagnosticsUpdate;
 import org.apache.tez.dag.app.dag.event.DAGEventSchedulerUpdate;
 import org.apache.tez.dag.app.dag.event.DAGEventType;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventKillRequest;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventOutputFailed;
-import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.dag.app.dag.event.TaskEvent;
-import org.apache.tez.dag.app.dag.event.TaskEventRecoverTask;
 import org.apache.tez.dag.app.dag.event.TaskEventScheduleTask;
 import org.apache.tez.dag.app.dag.event.TaskEventTAUpdate;
 import org.apache.tez.dag.app.dag.event.TaskEventTermination;
@@ -84,9 +82,6 @@ import org.apache.tez.dag.app.dag.event.VertexEventTaskReschedule;
 import org.apache.tez.dag.app.rm.container.AMContainer;
 import org.apache.tez.dag.app.rm.node.AMNodeEventTaskAttemptEnded;
 import org.apache.tez.dag.history.DAGHistoryEvent;
-import org.apache.tez.dag.history.HistoryEvent;
-import org.apache.tez.dag.history.events.TaskAttemptFinishedEvent;
-import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
 import org.apache.tez.dag.history.events.TaskFinishedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
@@ -138,6 +133,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   long scheduledTime;
   final StateChangeNotifier stateChangeNotifier;
 
+  private final TaskRecoveryData recoveryData;
+
   private final List<TezEvent> tezEventsForTaskAttempts = new ArrayList<TezEvent>();
   static final ArrayList<TezEvent> EMPTY_TASK_ATTEMPT_TEZ_EVENTS =
       new ArrayList(0);
@@ -150,8 +147,6 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   private static final SingleArcTransition<TaskImpl, TaskEvent>
      KILL_TRANSITION = new KillTransition();
 
-  // Recovery related flags
-  boolean recoveryStartEventSeen = false;
 
   private static final TaskStateChangedCallback STATE_CHANGED_CALLBACK = new TaskStateChangedCallback();
   
@@ -164,20 +159,14 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     // define the state machine of Task
 
     // Transitions from NEW state
-    .addTransition(TaskStateInternal.NEW, TaskStateInternal.SCHEDULED,
+    // Stay in NEW in recovery when Task is killed in the previous AM           
+    .addTransition(TaskStateInternal.NEW,
+        EnumSet.of(TaskStateInternal.NEW, TaskStateInternal.SCHEDULED),
         TaskEventType.T_SCHEDULE, new InitialScheduleTransition())
     .addTransition(TaskStateInternal.NEW, TaskStateInternal.KILLED,
         TaskEventType.T_TERMINATE,
         new KillNewTransition())
 
-    // Recover transition
-    .addTransition(TaskStateInternal.NEW,
-        EnumSet.of(TaskStateInternal.NEW,
-            TaskStateInternal.SCHEDULED,
-            TaskStateInternal.RUNNING, TaskStateInternal.SUCCEEDED,
-            TaskStateInternal.FAILED, TaskStateInternal.KILLED),
-        TaskEventType.T_RECOVER, new RecoverTransition())
-
     // Transitions from SCHEDULED state
       //when the first attempt is launched, the task state is set to RUNNING
      .addTransition(TaskStateInternal.SCHEDULED, TaskStateInternal.RUNNING,
@@ -191,6 +180,11 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
         EnumSet.of(TaskStateInternal.SCHEDULED, TaskStateInternal.FAILED),
         TaskEventType.T_ATTEMPT_FAILED,
         new AttemptFailedTransition())
+     // Happens in recovery   
+     .addTransition(TaskStateInternal.SCHEDULED,
+        EnumSet.of(TaskStateInternal.RUNNING, TaskStateInternal.SUCCEEDED),
+        TaskEventType.T_ATTEMPT_SUCCEEDED,
+        new AttemptSucceededTransition())
 
     // When current attempt fails/killed and new attempt launched then
     // TODO Task should go back to SCHEDULED state TEZ-495
@@ -199,7 +193,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
         TaskEventType.T_ATTEMPT_LAUNCHED) //more attempts may start later
     .addTransition(TaskStateInternal.RUNNING, TaskStateInternal.RUNNING,
         TaskEventType.T_ADD_SPEC_ATTEMPT, new RedundantScheduleTransition())
-    .addTransition(TaskStateInternal.RUNNING, TaskStateInternal.SUCCEEDED,
+    .addTransition(TaskStateInternal.RUNNING, 
+        EnumSet.of(TaskStateInternal.SUCCEEDED),
         TaskEventType.T_ATTEMPT_SUCCEEDED,
         new AttemptSucceededTransition())
     .addTransition(TaskStateInternal.RUNNING, TaskStateInternal.RUNNING,
@@ -327,7 +322,6 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   int failedAttempts;
 
   private final boolean leafVertex;
-  private TaskState recoveredState = TaskState.NEW;
 
   @Override
   public TaskState getState() {
@@ -366,6 +360,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     this.leafVertex = leafVertex;
     this.taskResource = resource;
     this.containerContext = containerContext;
+    this.recoveryData = appContext.getDAGRecoveryData() == null ?
+        null : appContext.getDAGRecoveryData().getTaskRecoveryData(taskId);
     stateMachine = new StateMachineTez<TaskStateInternal, TaskEventType, TaskEvent, TaskImpl>(
         stateMachineFactory.make(this), this);
     augmentStateMachine();
@@ -545,122 +541,6 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     }
   }
 
-  private TaskAttempt createRecoveredTaskAttempt(TezTaskAttemptID tezTaskAttemptID) {
-    TaskAttempt taskAttempt = createAttempt(tezTaskAttemptID.getId(), null);
-    return taskAttempt;
-  }
-
-  @Override
-  public TaskState restoreFromEvent(HistoryEvent historyEvent) {
-    writeLock.lock();
-    try {
-      switch (historyEvent.getEventType()) {
-        case TASK_STARTED:
-        {
-          TaskStartedEvent tEvent = (TaskStartedEvent) historyEvent;
-          recoveryStartEventSeen = true;
-          this.scheduledTime = tEvent.getScheduledTime();
-          if (this.attempts == null
-              || this.attempts.isEmpty()) {
-            this.attempts = new LinkedHashMap<TezTaskAttemptID, TaskAttempt>();
-          }
-          recoveredState = TaskState.SCHEDULED;
-          taskAttemptStatus.clear();
-          return recoveredState;
-        }
-        case TASK_FINISHED:
-        {
-          TaskFinishedEvent tEvent = (TaskFinishedEvent) historyEvent;
-          if (!recoveryStartEventSeen
-              && !tEvent.getState().equals(TaskState.KILLED)) {
-            throw new TezUncheckedException("Finished Event seen but"
-                + " no Started Event was encountered earlier"
-                + ", taskId=" + taskId
-                + ", finishState=" + tEvent.getState());
-          }
-          recoveredState = tEvent.getState();
-          if (tEvent.getState() == TaskState.SUCCEEDED
-              && tEvent.getSuccessfulAttemptID() != null) {
-            successfulAttempt = tEvent.getSuccessfulAttemptID();
-          }
-          return recoveredState;
-        }
-        case TASK_ATTEMPT_STARTED:
-        {
-          TaskAttemptStartedEvent taskAttemptStartedEvent =
-              (TaskAttemptStartedEvent) historyEvent;
-          TaskAttempt recoveredAttempt = createRecoveredTaskAttempt(
-              taskAttemptStartedEvent.getTaskAttemptID());
-          recoveredAttempt.restoreFromEvent(taskAttemptStartedEvent);
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Adding restored attempt into known attempts map"
-                + ", taskAttemptId=" + taskAttemptStartedEvent.getTaskAttemptID());
-          }
-          Preconditions.checkArgument(this.attempts.put(taskAttemptStartedEvent.getTaskAttemptID(),
-              recoveredAttempt) == null, taskAttemptStartedEvent.getTaskAttemptID() + " already existed.");
-          this.taskAttemptStatus.put(taskAttemptStartedEvent.getTaskAttemptID().getId(), false);
-          this.recoveredState = TaskState.RUNNING;
-          return recoveredState;
-        }
-        case TASK_ATTEMPT_FINISHED:
-        {
-          TaskAttemptFinishedEvent taskAttemptFinishedEvent =
-              (TaskAttemptFinishedEvent) historyEvent;
-          TaskAttempt taskAttempt = this.attempts.get(
-              taskAttemptFinishedEvent.getTaskAttemptID());
-          this.taskAttemptStatus.put(taskAttemptFinishedEvent.getTaskAttemptID().getId(), true);
-          if (taskAttempt == null) {
-            LOG.warn("Received an attempt finished event for an attempt that "
-                + " never started or does not exist"
-                + ", taskAttemptId=" + taskAttemptFinishedEvent.getTaskAttemptID()
-                + ", taskAttemptFinishState=" + taskAttemptFinishedEvent.getState());
-            TaskAttempt recoveredAttempt = createRecoveredTaskAttempt(
-                taskAttemptFinishedEvent.getTaskAttemptID());
-            this.attempts.put(taskAttemptFinishedEvent.getTaskAttemptID(),
-                recoveredAttempt);
-            // Allow TaskAttemptFinishedEvent without TaskAttemptStartedEvent when it is KILLED/FAILED
-            if (!taskAttemptFinishedEvent.getState().equals(TaskAttemptState.KILLED)
-                && !taskAttemptFinishedEvent.getState().equals(TaskAttemptState.FAILED)) {
-              throw new TezUncheckedException("Could not find task attempt"
-                  + " when trying to recover"
-                  + ", taskAttemptId=" + taskAttemptFinishedEvent.getTaskAttemptID()
-                  + ", taskAttemptFinishState" + taskAttemptFinishedEvent.getState());
-            }
-            taskAttempt = recoveredAttempt;
-          }
-          if (getUncompletedAttemptsCount() < 0) {
-            throw new TezUncheckedException("Invalid recovery event for attempt finished"
-                + ", more completions than starts encountered"
-                + ", taskId=" + taskId
-                + ", finishedAttempts=" + getFinishedAttemptsCount()
-                + ", incompleteAttempts=" + getUncompletedAttemptsCount());
-          }
-          TaskAttemptState taskAttemptState = taskAttempt.restoreFromEvent(
-              taskAttemptFinishedEvent);
-          if (taskAttemptState.equals(TaskAttemptState.SUCCEEDED)) {
-            recoveredState = TaskState.SUCCEEDED;
-            successfulAttempt = taskAttempt.getID();
-          } else if (taskAttemptState.equals(TaskAttemptState.FAILED)){
-            failedAttempts++;
-            getVertex().incrementFailedTaskAttemptCount();
-            successfulAttempt = null;
-            recoveredState = TaskState.RUNNING; // reset to RUNNING, may fail after SUCCEEDED
-          } else if (taskAttemptState.equals(TaskAttemptState.KILLED)) {
-            successfulAttempt = null;
-            getVertex().incrementKilledTaskAttemptCount();
-            recoveredState = TaskState.RUNNING; // reset to RUNNING, may been killed after SUCCEEDED
-          }
-          return recoveredState;
-        }
-        default:
-          throw new RuntimeException("Unexpected event received for restoring"
-              + " state, eventType=" + historyEvent.getEventType());
-      }
-    } finally {
-      writeLock.unlock();
-    }
-  }
-
   @VisibleForTesting
   public TaskStateInternal getInternalState() {
     readLock.lock();
@@ -1046,17 +926,39 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   }
 
   private static class InitialScheduleTransition
-    implements SingleArcTransition<TaskImpl, TaskEvent> {
+    implements MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
 
     @Override
-    public void transition(TaskImpl task, TaskEvent event) {
+    public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
+      if (task.recoveryData != null) {
+        TaskStartedEvent tStartedEvent = task.recoveryData.getTaskStartedEvent();
+        TaskFinishedEvent tFinishedEvent = task.recoveryData.getTaskFinishedEvent();
+        // If TaskStartedEvent is not seen but TaskFinishedEvent is seen, that means 
+        // Task is killed before it is started. Just send T_TERMINATE to itself to move to KILLED
+        if (tStartedEvent == null
+            && tFinishedEvent != null) {
+          Preconditions.checkArgument(tFinishedEvent.getState() == TaskState.KILLED,
+              "TaskStartedEvent is not seen, but TaskFinishedEvent is seen and with invalid state="
+                  + tFinishedEvent.getState() + ", taskId=" + task.getTaskId());
+          // TODO (TEZ-2938)
+          // use tFinishedEvent.getTerminationCause after adding TaskTerminationCause to TaskFinishedEvent
+          task.eventHandler.handle(new TaskEventTermination(task.taskId,
+              TaskAttemptTerminationCause.UNKNOWN_ERROR, tFinishedEvent.getDiagnostics(), true));
+          return TaskStateInternal.NEW;
+        }
+      } else {
+        task.scheduledTime = task.clock.getTime();
+        task.logJobHistoryTaskStartedEvent();
+      }
+      // No matter whether it is in recovery or normal execution, always schedule new task attempt.
+      // TaskAttempt will continue the recovery if necessary and send task attempt status
+      // to this Task.
       TaskEventScheduleTask scheduleEvent = (TaskEventScheduleTask) event;
       task.locationHint = scheduleEvent.getTaskLocationHint();
       task.baseTaskSpec = scheduleEvent.getBaseTaskSpec();
       // For now, initial scheduling dependency is due to vertex manager scheduling
       task.addAndScheduleAttempt(null);
-      task.scheduledTime = task.clock.getTime();
-      task.logJobHistoryTaskStartedEvent();
+      return TaskStateInternal.SCHEDULED;
     }
   }
 
@@ -1085,10 +987,67 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
 
 
   private static class AttemptSucceededTransition
-      implements SingleArcTransition<TaskImpl, TaskEvent> {
+      implements MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
+
+    private boolean recoverSuccessTaskAttempt(TaskImpl task) {
+      // Found successful attempt
+      // Recover data
+      boolean recoveredData = true;
+      if (task.getVertex().getOutputCommitters() != null
+          && !task.getVertex().getOutputCommitters().isEmpty()) {
+        for (Entry<String, OutputCommitter> entry
+            : task.getVertex().getOutputCommitters().entrySet()) {
+          LOG.info("Recovering data for task from previous DAG attempt"
+              + ", taskId=" + task.getTaskId()
+              + ", output=" + entry.getKey());
+          OutputCommitter committer = entry.getValue();
+          if (!committer.isTaskRecoverySupported()) {
+            LOG.info("Task recovery not supported by committer"
+                + ", failing task attempt"
+                + ", taskId=" + task.getTaskId()
+                + ", attemptId=" + task.successfulAttempt
+                + ", output=" + entry.getKey());
+            recoveredData = false;
+            break;
+          }
+          try {
+            committer.recoverTask(task.getTaskId().getId(),
+                task.appContext.getApplicationAttemptId().getAttemptId()-1);
+          } catch (Exception e) {
+            LOG.warn("Task recovery failed by committer"
+                + ", taskId=" + task.getTaskId()
+                + ", attemptId=" + task.successfulAttempt
+                + ", output=" + entry.getKey(), e);
+            recoveredData = false;
+            break;
+          }
+        }
+      }
+      return recoveredData;
+    }
+
     @Override
-    public void transition(TaskImpl task, TaskEvent event) {
+    public TaskStateInternal transition(TaskImpl task, TaskEvent event) {
       TezTaskAttemptID successTaId = ((TaskEventTAUpdate) event).getTaskAttemptID();
+      // Try to recover the succeeded TaskAttempt. It may be not recoverable if has committer which don't support
+      // recovery. In that case just reschedule new attempt if numFailedAttempts does not exceeded maxFailedAttempts.
+      if (task.recoveryData!= null
+          && task.recoveryData.isTaskAttemptSucceeded(successTaId)) {
+        boolean recoveredData = recoverSuccessTaskAttempt(task);
+        if (!recoveredData) {
+          // Move this TA to KILLED (TEZ-2958)
+          LOG.info("Can not recovery the successful task attempt, schedule new task attempt,"
+              + "taskId=" + task.getTaskId());
+          task.successfulAttempt = null;
+          task.addAndScheduleAttempt(successTaId);
+          return TaskStateInternal.RUNNING;
+        } else {
+          task.successfulAttempt = successTaId;
+          LOG.info("Recovered a successful attempt"
+              + ", taskAttemptId=" + task.successfulAttempt.toString());
+        }
+      }
+      // both recovery to succeeded and normal dag succeeded go here.
       if (task.commitAttempt != null &&
           !task.commitAttempt.equals(successTaId)) {
         // The succeeded attempt is not the one that was selected to commit
@@ -1136,7 +1095,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       task.eventHandler.handle(new DAGEventSchedulerUpdate(
           DAGEventSchedulerUpdate.UpdateType.TA_SUCCEEDED, task.attempts
               .get(task.successfulAttempt)));
-      task.finished(TaskStateInternal.SUCCEEDED);
+      return task.finished(TaskStateInternal.SUCCEEDED);
     }
   }
 
@@ -1162,139 +1121,6 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     }
   }
 
-  private static class RecoverTransition implements
-      MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
-
-    @Override
-    public TaskStateInternal transition(TaskImpl task, TaskEvent taskEvent) {
-      if (taskEvent instanceof TaskEventRecoverTask) {
-        TaskEventRecoverTask taskEventRecoverTask =
-            (TaskEventRecoverTask) taskEvent;
-        if (taskEventRecoverTask.getDesiredState() != null
-            && !taskEventRecoverTask.recoverData()) {
-          // TODO recover attempts if desired state is given?
-          // History may not have all data.
-          switch (taskEventRecoverTask.getDesiredState()) {
-            case SUCCEEDED:
-              return TaskStateInternal.SUCCEEDED;
-            case FAILED:
-              return TaskStateInternal.FAILED;
-            case KILLED:
-              return TaskStateInternal.KILLED;
-          }
-        }
-      }
-
-      TaskStateInternal endState = TaskStateInternal.NEW;
-      if (task.attempts != null) {
-        for (TaskAttempt taskAttempt : task.attempts.values()) {
-          task.eventHandler.handle(new TaskAttemptEvent(
-              taskAttempt.getID(), TaskAttemptEventType.TA_RECOVER));
-        }
-      }
-      LOG.info("Trying to recover task"
-          + ", taskId=" + task.getTaskId()
-          + ", recoveredState=" + task.recoveredState);
-      switch(task.recoveredState) {
-        case NEW:
-          // Nothing to do until the vertex schedules this task
-          endState = TaskStateInternal.NEW;
-          break;
-        case SCHEDULED:
-        case RUNNING:
-        case SUCCEEDED:
-          if (task.successfulAttempt != null) {
-            //Found successful attempt
-            //Recover data
-            boolean recoveredData = true;
-            if (task.getVertex().getOutputCommitters() != null
-                && !task.getVertex().getOutputCommitters().isEmpty()) {
-              for (Entry<String, OutputCommitter> entry
-                  : task.getVertex().getOutputCommitters().entrySet()) {
-                LOG.info("Recovering data for task from previous DAG attempt"
-                    + ", taskId=" + task.getTaskId()
-                    + ", output=" + entry.getKey());
-                OutputCommitter committer = entry.getValue();
-                if (!committer.isTaskRecoverySupported()) {
-                  LOG.info("Task recovery not supported by committer"
-                      + ", failing task attempt"
-                      + ", taskId=" + task.getTaskId()
-                      + ", attemptId=" + task.successfulAttempt
-                      + ", output=" + entry.getKey());
-                  recoveredData = false;
-                  break;
-                }
-                try {
-                  committer.recoverTask(task.getTaskId().getId(),
-                      task.appContext.getApplicationAttemptId().getAttemptId()-1);
-                } catch (Exception e) {
-                  LOG.warn("Task recovery failed by committer"
-                      + ", taskId=" + task.getTaskId()
-                      + ", attemptId=" + task.successfulAttempt
-                      + ", output=" + entry.getKey(), e);
-                  recoveredData = false;
-                  break;
-                }
-              }
-            }
-            if (!recoveredData) {
-              task.successfulAttempt = null;
-            } else {
-              LOG.info("Recovered a successful attempt"
-                  + ", taskAttemptId=" + task.successfulAttempt.toString());
-              task.logJobHistoryTaskFinishedEvent();
-              task.eventHandler.handle(
-                  new VertexEventTaskCompleted(task.taskId,
-                      getExternalState(TaskStateInternal.SUCCEEDED)));
-              task.eventHandler.handle(
-                  new VertexEventTaskAttemptCompleted(
-                      task.successfulAttempt, TaskAttemptStateInternal.SUCCEEDED));
-              endState = TaskStateInternal.SUCCEEDED;
-              break;
-            }
-          }
-
-          if (endState != TaskStateInternal.SUCCEEDED &&
-              task.failedAttempts >= task.maxFailedAttempts) {
-            // Exceeded max attempts
-            task.finished(TaskStateInternal.FAILED);
-            endState = TaskStateInternal.FAILED;
-            break;
-          }
-
-          // no successful attempt and all attempts completed
-          // schedule a new one
-          // If any incomplete, the running attempt will moved to failed and its
-          // update will trigger a new attempt if possible
-          if (task.attempts.size() == task.getFinishedAttemptsCount()) {
-            task.addAndScheduleAttempt(null);
-          }
-          endState = TaskStateInternal.RUNNING;
-          break;
-        case KILLED:
-          // Nothing to do
-          // Inform vertex
-          task.eventHandler.handle(
-              new VertexEventTaskCompleted(task.taskId,
-                  getExternalState(TaskStateInternal.KILLED)));
-          endState  = TaskStateInternal.KILLED;
-          break;
-        case FAILED:
-          // Nothing to do
-          // Inform vertex
-          task.eventHandler.handle(
-              new VertexEventTaskCompleted(task.taskId,
-                  getExternalState(TaskStateInternal.FAILED)));
-
-          endState = TaskStateInternal.FAILED;
-          break;
-      }
-
-      return endState;
-    }
-  }
-
-
   private static class KillWaitAttemptCompletedTransition implements
       MultipleArcTransition<TaskImpl, TaskEvent, TaskStateInternal> {
 
@@ -1486,7 +1312,13 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     public void transition(TaskImpl task, TaskEvent event) {
       TaskEventTermination terminateEvent = (TaskEventTermination)event;
       task.addDiagnosticInfo(terminateEvent.getDiagnosticInfo());
-      task.logJobHistoryTaskFailedEvent(TaskState.KILLED);
+      if (terminateEvent.isFromRecovery()) {
+        if (LOG.isDebugEnabled()) {
+          LOG.debug("Recovered to KILLED, taskId=" + task.getTaskId());
+        }
+      } else {
+        task.logJobHistoryTaskFailedEvent(TaskState.KILLED);
+      }
       task.eventHandler.handle(
           new VertexEventTaskCompleted(task.taskId, TaskState.KILLED));
       // TODO Metrics


[9/9] tez git commit: TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)

Posted by zj...@apache.org.
TEZ-2581. Umbrella for Tez Recovery Redesign (zjffdu)


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

Branch: refs/heads/master
Commit: 28f30b0ef654f124713acb7a213e37bbc7d8b486
Parents: c4487f9
Author: Jeff Zhang <zj...@apache.org>
Authored: Wed Nov 25 22:01:44 2015 +0800
Committer: Jeff Zhang <zj...@apache.org>
Committed: Wed Nov 25 22:01:44 2015 +0800

----------------------------------------------------------------------
 CHANGES.txt                                     |    1 +
 .../apache/tez/dag/api/TezConfiguration.java    |    8 +
 .../apache/tez/dag/api/event/VertexState.java   |    7 +-
 .../records/TaskAttemptTerminationCause.java    |    1 +
 .../tez/dag/api/client/VertexStatusBuilder.java |    2 -
 .../java/org/apache/tez/dag/app/AppContext.java |    4 +
 .../org/apache/tez/dag/app/DAGAppMaster.java    |   38 +-
 .../org/apache/tez/dag/app/RecoveryParser.java  |  721 ++++++---
 .../tez/dag/app/TaskCommunicatorManager.java    |   79 +-
 .../java/org/apache/tez/dag/app/dag/DAG.java    |    3 -
 .../java/org/apache/tez/dag/app/dag/Task.java   |    3 -
 .../org/apache/tez/dag/app/dag/TaskAttempt.java |    3 -
 .../java/org/apache/tez/dag/app/dag/Vertex.java |    3 -
 .../org/apache/tez/dag/app/dag/VertexState.java |    1 -
 .../dag/app/dag/event/DAGEventRecoverEvent.java |   23 +-
 .../tez/dag/app/dag/event/RecoveryEvent.java    |   23 +
 .../event/TaskAttemptEventAttemptFailed.java    |   15 +-
 .../event/TaskAttemptEventAttemptKilled.java    |   16 +-
 .../dag/event/TaskAttemptEventKillRequest.java  |   14 +-
 .../event/TaskAttemptEventStartedRemotely.java  |   14 +-
 .../event/TaskAttemptEventTezEventUpdate.java   |   37 +
 .../dag/app/dag/event/TaskAttemptEventType.java |    4 +-
 .../dag/app/dag/event/TaskEventRecoverTask.java |   53 -
 .../app/dag/event/TaskEventScheduleTask.java    |   14 +-
 .../dag/app/dag/event/TaskEventTermination.java |   16 +-
 .../tez/dag/app/dag/event/TaskEventType.java    |    3 -
 .../app/dag/event/VertexEventRecoverVertex.java |    1 -
 .../app/dag/event/VertexEventRouteEvent.java    |   12 -
 .../event/VertexEventSourceVertexRecovered.java |   62 -
 .../tez/dag/app/dag/event/VertexEventType.java  |    3 -
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |  364 ++---
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |  273 ++--
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |  384 ++---
 .../apache/tez/dag/app/dag/impl/VertexImpl.java | 1426 ++++------------
 .../tez/dag/history/HistoryEventHandler.java    |    5 +-
 .../tez/dag/history/HistoryEventType.java       |    3 +-
 .../tez/dag/history/RecoveryConverters.java     |   27 +
 .../events/TaskAttemptFinishedEvent.java        |   26 +-
 .../events/VertexConfigurationDoneEvent.java    |  211 +++
 .../events/VertexGroupCommitFinishedEvent.java  |   26 +-
 .../events/VertexGroupCommitStartedEvent.java   |   25 +-
 .../history/events/VertexInitializedEvent.java  |   35 +-
 .../events/VertexParallelismUpdatedEvent.java   |  204 ---
 .../VertexRecoverableEventsGeneratedEvent.java  |  224 ---
 .../impl/HistoryEventJsonConversion.java        |   81 +-
 .../dag/history/recovery/RecoveryService.java   |   10 +-
 .../tez/dag/history/utils/TezEventUtils.java    |  131 ++
 tez-dag/src/main/proto/HistoryEvents.proto      |   31 +-
 .../dag/api/client/TestVertexStatusBuilder.java |    7 +-
 .../apache/tez/dag/app/TestRecoveryParser.java  |  480 +++++-
 .../dag/app/TestTaskCommunicatorManager1.java   |   56 +-
 .../tez/dag/app/dag/impl/TestDAGRecovery.java   | 1527 ++++++++++++------
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   |    5 +-
 .../app/dag/impl/TestTaskAttemptRecovery.java   |  327 ----
 .../tez/dag/app/dag/impl/TestTaskImpl.java      |    5 +-
 .../tez/dag/app/dag/impl/TestTaskRecovery.java  |  873 ----------
 .../tez/dag/app/dag/impl/TestVertexImpl.java    |   33 +-
 .../dag/app/dag/impl/TestVertexRecovery.java    | 1340 ---------------
 .../TestHistoryEventsProtoConversion.java       |  220 ++-
 .../impl/TestHistoryEventJsonConversion.java    |   27 +-
 .../org/apache/tez/examples/TezExampleBase.java |   12 +
 .../ats/HistoryEventTimelineConversion.java     |   18 +-
 .../ats/TestHistoryEventTimelineConversion.java |   29 +-
 .../apache/tez/runtime/api/impl/TezEvent.java   |    2 +-
 .../apache/tez/test/AMShutdownController.java   |   57 +
 .../RecoveryServiceWithEventHandlingHook.java   |  386 +++++
 .../org/apache/tez/test/TestDAGRecovery.java    |   62 -
 .../java/org/apache/tez/test/TestRecovery.java  |  484 ++++++
 .../java/org/apache/tez/test/TestTezJobs.java   |    6 +-
 .../apache/tez/test/dag/MultiAttemptDAG.java    |    2 +-
 70 files changed, 4639 insertions(+), 5989 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 59847ef..b9a91f8 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
   TEZ-2949. Allow duplicate dag names within session for Tez.
 
 ALL CHANGES:
+  TEZ-2581. Umbrella for Tez Recovery Redesign
   TEZ-2956. Handle auto-reduce parallelism when the
   totalNumBipartiteSourceTasks is 0
   TEZ-2947. Tez UI: Timeline, RM & AM requests gets into a consecutive loop in counters page without any delay

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
index 0ea8999..fabc256 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/TezConfiguration.java
@@ -1493,4 +1493,12 @@ public class TezConfiguration extends Configuration {
   @ConfigurationProperty(type="boolean")
   public static final String TEZ_CLIENT_ASYNCHRONOUS_STOP = TEZ_PREFIX + "client.asynchronous-stop";
   public static final boolean TEZ_CLIENT_ASYNCHRONOUS_STOP_DEFAULT = true;
+
+  // for Recovery Test
+  @Private
+  @ConfigurationScope(Scope.TEST)
+  public static final String TEZ_AM_RECOVERY_SERVICE_CLASS =
+      TEZ_PREFIX + "test.recovery-service-class";
+  @Private
+  public static final String TEZ_AM_RECOVERY_SERVICE_CLASS_DEFAULT = "org.apache.tez.dag.history.recovery.RecoveryService";
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-api/src/main/java/org/apache/tez/dag/api/event/VertexState.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/event/VertexState.java b/tez-api/src/main/java/org/apache/tez/dag/api/event/VertexState.java
index c9c2d58..86e70a1 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/event/VertexState.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/event/VertexState.java
@@ -56,5 +56,10 @@ public enum VertexState {
    * further. Listeners can depend on the vertex's configured state after
    * receiving this notification.
    */
-  CONFIGURED
+  CONFIGURED,
+
+  /**
+   * Indicates that the Vertex move to INITIALIZING
+   */
+  INITIALIZING
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
----------------------------------------------------------------------
diff --git a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
index a5214fb..14eaa3a 100644
--- a/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
+++ b/tez-common/src/main/java/org/apache/tez/dag/records/TaskAttemptTerminationCause.java
@@ -23,6 +23,7 @@ public enum TaskAttemptTerminationCause {
   
   TERMINATED_BY_CLIENT, // Killed by client command
   TERMINATED_AT_SHUTDOWN, // Killed due execution shutdown
+  TERMINATED_AT_RECOVERY, // Killed in recovery, due to can not recover running task attempt
   INTERNAL_PREEMPTION, // Killed by Tez to makes space for higher pri work
   EXTERNAL_PREEMPTION, // Killed by the cluster to make space for other work
   TERMINATED_INEFFECTIVE_SPECULATION, // Killed speculative attempt because original succeeded

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java
index ada3490..4de321c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/VertexStatusBuilder.java
@@ -65,8 +65,6 @@ public class VertexStatusBuilder extends VertexStatus {
         return VertexStatusStateProto.VERTEX_NEW;
       case INITIALIZING:
         return VertexStatusStateProto.VERTEX_INITIALIZING;
-      case RECOVERING:
-        return VertexStatusStateProto.VERTEX_NEW;
       case INITED:
         return VertexStatusStateProto.VERTEX_INITED;
       case RUNNING:

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
index 68453b1..30716da 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/AppContext.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.rm.TaskSchedulerManager;
 import org.apache.tez.dag.app.rm.container.AMContainerMap;
@@ -76,6 +77,8 @@ public interface AppContext {
 
   void setDAG(DAG dag);
 
+  void setDAGRecoveryData(DAGRecoveryData dagRecoveryData);
+
   Set<String> getAllDAGIDs();
 
   @SuppressWarnings("rawtypes")
@@ -126,4 +129,5 @@ public interface AppContext {
 
   public HadoopShim getHadoopShim();
 
+  public DAGRecoveryData getDAGRecoveryData();
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
index 2c50264..23981e7 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/DAGAppMaster.java
@@ -127,7 +127,7 @@ import org.apache.tez.dag.api.records.DAGProtos;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.dag.api.records.DAGProtos.PlanLocalResourcesProto;
 import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
-import org.apache.tez.dag.app.RecoveryParser.RecoveredDAGData;
+import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.DAGState;
 import org.apache.tez.dag.app.dag.Task;
@@ -151,6 +151,10 @@ import org.apache.tez.dag.app.dag.event.VertexEvent;
 import org.apache.tez.dag.app.dag.event.VertexEventType;
 import org.apache.tez.dag.app.dag.impl.DAGImpl;
 import org.apache.tez.dag.app.launcher.ContainerLauncherManager;
+import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
+import org.apache.tez.dag.app.dag.impl.TaskImpl;
+import org.apache.tez.dag.app.dag.impl.VertexImpl;
+import org.apache.tez.dag.app.launcher.LocalContainerLauncher;
 import org.apache.tez.dag.app.rm.AMSchedulerEventType;
 import org.apache.tez.dag.app.rm.ContainerLauncherEventType;
 import org.apache.tez.dag.app.rm.TaskSchedulerManager;
@@ -1417,6 +1421,7 @@ public class DAGAppMaster extends AbstractService {
   private class RunningAppContext implements AppContext {
 
     private DAG dag;
+    private DAGRecoveryData dagRecoveryData;
     private final Configuration conf;
     private final ClusterInfo clusterInfo = new ClusterInfo();
     private final ReentrantReadWriteLock rwLock = new ReentrantReadWriteLock();
@@ -1633,6 +1638,7 @@ public class DAGAppMaster extends AbstractService {
       try {
         wLock.lock();
         this.dag = dag;
+        this.dagRecoveryData = null;
       } finally {
         wLock.unlock();
       }
@@ -1647,6 +1653,16 @@ public class DAGAppMaster extends AbstractService {
     public long getCumulativeGCTime() {
       return getAMGCTime();
     }
+
+    @Override
+    public void setDAGRecoveryData(DAGRecoveryData dagRecoveryData) {
+      this.dagRecoveryData = dagRecoveryData;
+    }
+
+    @Override
+    public DAGRecoveryData getDAGRecoveryData() {
+      return dagRecoveryData;
+    }
   }
 
   private static class ServiceWithDependency implements ServiceStateChangeListener {
@@ -1818,7 +1834,7 @@ public class DAGAppMaster extends AbstractService {
     }
   }
 
-  private RecoveredDAGData recoverDAG() throws IOException, TezException {
+  private DAGRecoveryData recoverDAG() throws IOException, TezException {
     if (recoveryEnabled) {
       if (this.appAttemptID.getAttemptId() > 1) {
         LOG.info("Recovering data from previous attempts"
@@ -1826,7 +1842,7 @@ public class DAGAppMaster extends AbstractService {
         this.state = DAGAppMasterState.RECOVERING;
         RecoveryParser recoveryParser = new RecoveryParser(
             this, recoveryFS, recoveryDataDir, appAttemptID.getAttemptId());
-        RecoveredDAGData recoveredDAGData = recoveryParser.parseRecoveryData();
+        DAGRecoveryData recoveredDAGData = recoveryParser.parseRecoveryData();
         return recoveredDAGData;
       }
     }
@@ -1855,7 +1871,7 @@ public class DAGAppMaster extends AbstractService {
 
     this.lastDAGCompletionTime = clock.getTime();
 
-    RecoveredDAGData recoveredDAGData;
+    DAGRecoveryData recoveredDAGData;
     try {
       recoveredDAGData = recoverDAG();
     } catch (IOException e) {
@@ -1875,9 +1891,8 @@ public class DAGAppMaster extends AbstractService {
     }
 
     if (recoveredDAGData != null) {
-      List<URL> classpathUrls = null;
       if (recoveredDAGData.cumulativeAdditionalResources != null) {
-        classpathUrls = processAdditionalResources(recoveredDAGData.cumulativeAdditionalResources);
+        recoveredDAGData.additionalUrlsForClasspath = processAdditionalResources(recoveredDAGData.cumulativeAdditionalResources);
         amResources.putAll(recoveredDAGData.cumulativeAdditionalResources);
         cumulativeAdditionalResources.putAll(recoveredDAGData.cumulativeAdditionalResources);
       }
@@ -1900,9 +1915,11 @@ public class DAGAppMaster extends AbstractService {
             + ", failureReason=" + recoveredDAGData.reason);
         _updateLoggers(recoveredDAGData.recoveredDAG, "");
         if (recoveredDAGData.nonRecoverable) {
+          addDiagnostic("DAG " + recoveredDAGData.recoveredDagID + " can not be recovered due to "
+              + recoveredDAGData.reason);
           DAGEventRecoverEvent recoverDAGEvent =
               new DAGEventRecoverEvent(recoveredDAGData.recoveredDAG.getID(),
-                  DAGState.FAILED, classpathUrls);
+                  DAGState.FAILED, recoveredDAGData);
           DAGRecoveredEvent dagRecoveredEvent = new DAGRecoveredEvent(this.appAttemptID,
               recoveredDAGData.recoveredDAG.getID(), recoveredDAGData.recoveredDAG.getName(),
               recoveredDAGData.recoveredDAG.getUserName(),
@@ -1919,7 +1936,7 @@ public class DAGAppMaster extends AbstractService {
         } else {
           DAGEventRecoverEvent recoverDAGEvent =
               new DAGEventRecoverEvent(recoveredDAGData.recoveredDAG.getID(),
-                  recoveredDAGData.dagState, classpathUrls);
+                  recoveredDAGData.dagState, recoveredDAGData);
           DAGRecoveredEvent dagRecoveredEvent = new DAGRecoveredEvent(this.appAttemptID,
               recoveredDAGData.recoveredDAG.getID(), recoveredDAGData.recoveredDAG.getName(),
               recoveredDAGData.recoveredDAG.getUserName(), this.clock.getTime(),
@@ -1938,7 +1955,7 @@ public class DAGAppMaster extends AbstractService {
         this.historyEventHandler.handle(new DAGHistoryEvent(recoveredDAGData.recoveredDAG.getID(),
             dagRecoveredEvent));
         DAGEventRecoverEvent recoverDAGEvent = new DAGEventRecoverEvent(
-            recoveredDAGData.recoveredDAG.getID(), classpathUrls);
+            recoveredDAGData.recoveredDAG.getID(), recoveredDAGData);
         dagEventDispatcher.handle(recoverDAGEvent);
         this.state = DAGAppMasterState.RUNNING;
       }
@@ -2050,7 +2067,6 @@ public class DAGAppMaster extends AbstractService {
       if (dag == null || eventDagIndex != dag.getID().getId()) {
         return; // event not relevant any more
       }
-      
       Task task =
           dag.getVertex(event.getTaskID().getVertexID()).
               getTask(event.getTaskID());
@@ -2432,7 +2448,6 @@ public class DAGAppMaster extends AbstractService {
         TezConfiguration.TEZ_AM_WEBSERVICE_ENABLE_DEFAULT);
   }
 
-
   @VisibleForTesting
   static void parseAllPlugins(
       List<NamedEntityDescriptor> taskSchedulerDescriptors, BiMap<String, Integer> taskSchedulerPluginMap,
@@ -2547,4 +2562,5 @@ public class DAGAppMaster extends AbstractService {
     }
     return sb.toString();
   }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
index 046dbd9..368dd17 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/RecoveryParser.java
@@ -20,6 +20,7 @@ package org.apache.tez.dag.app;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.net.URL;
 import java.util.ArrayList;
 import java.util.HashMap;
 import java.util.List;
@@ -33,13 +34,14 @@ import org.apache.hadoop.fs.FSDataInputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.hadoop.yarn.api.records.LocalResource;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
+import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
 import org.apache.tez.dag.app.dag.DAGState;
-import org.apache.tez.dag.app.dag.Task;
-import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.dag.app.dag.VertexState;
 import org.apache.tez.dag.app.dag.impl.DAGImpl;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
@@ -58,21 +60,29 @@ import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
 import org.apache.tez.dag.history.events.TaskFinishedEvent;
 import org.apache.tez.dag.history.events.TaskStartedEvent;
 import org.apache.tez.dag.history.events.VertexCommitStartedEvent;
-import org.apache.tez.dag.history.events.VertexRecoverableEventsGeneratedEvent;
 import org.apache.tez.dag.history.events.VertexFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitFinishedEvent;
 import org.apache.tez.dag.history.events.VertexGroupCommitStartedEvent;
 import org.apache.tez.dag.history.events.VertexInitializedEvent;
-import org.apache.tez.dag.history.events.VertexParallelismUpdatedEvent;
+import org.apache.tez.dag.history.events.VertexConfigurationDoneEvent;
 import org.apache.tez.dag.history.events.VertexStartedEvent;
 import org.apache.tez.dag.history.recovery.RecoveryService;
 import org.apache.tez.dag.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.recovery.records.RecoveryProtos;
 import org.apache.tez.dag.recovery.records.RecoveryProtos.SummaryEventProto;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Preconditions;
 
+
+/**
+ * RecoverParser is mainly for Tez AM Recovery. It would read the recovery events. (summary & non-summary)
+ * 
+ */
 public class RecoveryParser {
 
   private static final Logger LOG = LoggerFactory.getLogger(RecoveryParser.class);
@@ -100,7 +110,8 @@ public class RecoveryParser {
     this.recoveryFS.mkdirs(currentAttemptRecoveryDataDir);
   }
 
-  public static class RecoveredDAGData {
+  public static class DAGRecoveryData {
+
     public TezDAGID recoveredDagID = null;
     public DAGImpl recoveredDAG = null;
     public DAGState dagState = null;
@@ -109,6 +120,119 @@ public class RecoveryParser {
     public boolean isSessionStopped = false;
     public String reason = null;
     public Map<String, LocalResource> cumulativeAdditionalResources = null;
+    public List<URL> additionalUrlsForClasspath = null;
+
+    public Map<TezVertexID, VertexRecoveryData> vertexRecoveryDataMap =
+        new HashMap<TezVertexID, RecoveryParser.VertexRecoveryData>();
+    private DAGInitializedEvent dagInitedEvent;
+    private DAGStartedEvent dagStartedEvent;
+    private DAGFinishedEvent dagFinishedEvent;
+
+    private Map<TezVertexID, Boolean> vertexCommitStatus =
+        new HashMap<TezVertexID, Boolean>();
+    private Map<String, Boolean> vertexGroupCommitStatus =
+        new HashMap<String, Boolean>();
+    private Map<TezVertexID, Boolean> vertexGroupMemberCommitStatus =
+        new HashMap<TezVertexID, Boolean>();
+
+    public DAGRecoveryData(DAGSummaryData dagSummaryData) {
+      if (dagSummaryData.completed) {
+        this.isCompleted = true;
+        this.dagState = dagSummaryData.dagState;
+      }
+      dagSummaryData.checkRecoverableSummary();
+      this.nonRecoverable = dagSummaryData.nonRecoverable;
+      this.reason = dagSummaryData.reason;
+      this.vertexCommitStatus = dagSummaryData.vertexCommitStatus;
+      this.vertexGroupCommitStatus = dagSummaryData.vertexGroupCommitStatus;
+      this.vertexGroupMemberCommitStatus = dagSummaryData.vertexGroupMemberCommitStatus;
+    }
+
+    // DAG is not recoverable if vertex has committer and has completed the commit (based on summary recovery events)
+    // but its full recovery events are not seen. (based on non-summary recovery events)
+    // Unrecoverable reason: vertex is committed we cannot rerun it and if vertex recovery events are not completed 
+    // we cannot run other vertices that may depend on this one. So we have to abort.
+    public void checkRecoverableNonSummary() {
+      // It is OK without full recovering events if the dag is completed based on summary event.
+      if (isCompleted) {
+        return;
+      }
+      for (Map.Entry<TezVertexID, Boolean> entry : vertexCommitStatus.entrySet()) {
+        // vertex has finished committing
+        TezVertexID vertexId = entry.getKey();
+        boolean commitFinished = entry.getValue();
+        if(commitFinished
+            && (!vertexRecoveryDataMap.containsKey(vertexId)
+            || vertexRecoveryDataMap.get(vertexId).getVertexFinishedEvent() == null)) {
+          this.nonRecoverable = true;
+          this.reason = "Vertex has been committed, but its full recovery events are not seen, vertexId="
+              + vertexId;
+          return;
+        }
+      }
+      for (Map.Entry<TezVertexID, Boolean> entry : vertexGroupMemberCommitStatus.entrySet()) {
+        // vertex has finished committing
+        TezVertexID vertexId = entry.getKey();
+        boolean commitFinished = entry.getValue();
+        if(commitFinished
+            && (!vertexRecoveryDataMap.containsKey(vertexId)
+            || vertexRecoveryDataMap.get(vertexId).getVertexFinishedEvent() == null)) {
+          this.nonRecoverable = true;
+          this.reason = "Vertex has been committed as member of vertex group"
+              + ", but its full recovery events are not seen, vertexId=" + vertexId;
+          return;
+        }
+      }
+    }
+
+    public DAGInitializedEvent getDAGInitializedEvent() {
+      return dagInitedEvent;
+    }
+
+    public DAGStartedEvent getDAGStartedEvent() {
+      return dagStartedEvent;
+    }
+
+    public DAGFinishedEvent getDAGFinishedEvent() {
+      return dagFinishedEvent;
+    }
+
+    public boolean isVertexGroupCommitted(String groupName) {
+      return vertexGroupCommitStatus.containsKey(groupName)
+          && vertexGroupCommitStatus.get(groupName);
+    }
+
+    public VertexRecoveryData getVertexRecoveryData(TezVertexID vertexId) {
+      return vertexRecoveryDataMap.get(vertexId);
+    }
+
+    public TaskRecoveryData getTaskRecoveryData(TezTaskID taskId) {
+      VertexRecoveryData vertexRecoveryData = getVertexRecoveryData(taskId.getVertexID());
+      if (vertexRecoveryData != null) {
+        return vertexRecoveryData.taskRecoveryDataMap.get(taskId);
+      } else {
+        return null;
+      }
+    }
+
+    public TaskAttemptRecoveryData getTaskAttemptRecoveryData(TezTaskAttemptID taId) {
+      TaskRecoveryData taskRecoveryData = getTaskRecoveryData(taId.getTaskID());
+      if (taskRecoveryData != null) {
+        return taskRecoveryData.taRecoveryDataMap.get(taId);
+      } else {
+        return null;
+      }
+    }
+
+    public VertexRecoveryData maybeCreateVertexRecoveryData(TezVertexID vertexId) {
+      VertexRecoveryData vRecoveryData = vertexRecoveryDataMap.get(vertexId);
+      if (vRecoveryData == null) {
+        vRecoveryData = new VertexRecoveryData(vertexCommitStatus.containsKey(vertexId)
+            ? vertexCommitStatus.get(vertexId) : false);
+        vertexRecoveryDataMap.put(vertexId, vRecoveryData);
+      }
+      return vRecoveryData;
+    }
   }
 
   private static void parseSummaryFile(FSDataInputStream inputStream)
@@ -178,12 +302,12 @@ public class RecoveryParser {
       case VERTEX_INITIALIZED:
         event = new VertexInitializedEvent();
         break;
+      case VERTEX_CONFIGURE_DONE:
+        event = new VertexConfigurationDoneEvent();
+        break;
       case VERTEX_STARTED:
         event = new VertexStartedEvent();
         break;
-      case VERTEX_PARALLELISM_UPDATED:
-        event = new VertexParallelismUpdatedEvent();
-        break;
       case VERTEX_COMMIT_STARTED:
         event = new VertexCommitStartedEvent();
         break;
@@ -208,18 +332,11 @@ public class RecoveryParser {
       case TASK_ATTEMPT_FINISHED:
         event = new TaskAttemptFinishedEvent();
         break;
-      case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-        event = new VertexRecoverableEventsGeneratedEvent();
-        break;
       default:
         throw new IOException("Invalid data found, unknown event type "
             + eventType);
 
     }
-    if (LOG.isDebugEnabled()) {
-      LOG.debug("Parsing event from input stream"
-          + ", eventType=" + eventType);
-    }
     try {
       event.fromProtoStream(inputStream);
     } catch (EOFException eof) {
@@ -233,10 +350,6 @@ public class RecoveryParser {
     return event;
   }
 
-
-
-
-
   public static List<HistoryEvent> parseDAGRecoveryFile(FSDataInputStream inputStream)
       throws IOException {
     List<HistoryEvent> historyEvents = new ArrayList<HistoryEvent>();
@@ -246,11 +359,45 @@ public class RecoveryParser {
         LOG.info("Reached end of stream");
         break;
       }
+      LOG.debug("Read HistoryEvent, eventType=" + historyEvent.getEventType() + ", event=" + historyEvent);
       historyEvents.add(historyEvent);
     }
     return historyEvents;
   }
 
+  public static List<HistoryEvent> readRecoveryEvents(TezConfiguration tezConf, ApplicationId appId,
+      int attempt) throws IOException {
+    Path tezSystemStagingDir =
+        TezCommonUtils.getTezSystemStagingPath(tezConf, appId.toString());
+    Path recoveryDataDir =
+        TezCommonUtils.getRecoveryPath(tezSystemStagingDir, tezConf);
+    FileSystem fs = tezSystemStagingDir.getFileSystem(tezConf);
+    List<HistoryEvent> historyEvents = new ArrayList<HistoryEvent>();
+    for (int i=1; i <= attempt; ++i) {
+      Path currentAttemptRecoveryDataDir =
+          TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir, i);
+      Path recoveryFilePath =
+          new Path(currentAttemptRecoveryDataDir, appId.toString().replace(
+              "application", "dag")
+              + "_1" + TezConstants.DAG_RECOVERY_RECOVER_FILE_SUFFIX);
+      if (fs.exists(recoveryFilePath)) {
+        LOG.info("Read recovery file:" + recoveryFilePath);
+        FSDataInputStream in = null;
+        try {
+          in = fs.open(recoveryFilePath);
+          historyEvents.addAll(RecoveryParser.parseDAGRecoveryFile(in));
+        } catch (IOException e) {
+          throw e;
+        } finally {
+          if (in != null) {
+            in.close();
+          }
+        }
+      }
+    }
+    return historyEvents;
+  }
+
   public static void main(String argv[]) throws IOException {
     // TODO clean up with better usage and error handling
     Configuration conf = new Configuration();
@@ -325,13 +472,15 @@ public class RecoveryParser {
     final TezDAGID dagId;
     boolean completed = false;
     boolean dagCommitCompleted = true;
+    boolean nonRecoverable = false;
+    String reason;
     DAGState dagState;
-    Map<TezVertexID, Boolean> vertexCommitStatus =
+    public Map<TezVertexID, Boolean> vertexCommitStatus =
         new HashMap<TezVertexID, Boolean>();
-    Map<String, Boolean> vertexGroupCommitStatus =
+    public Map<String, Boolean> vertexGroupCommitStatus =
         new HashMap<String, Boolean>();
-    List<HistoryEvent> bufferedSummaryEvents =
-        new ArrayList<HistoryEvent>();
+    public Map<TezVertexID, Boolean> vertexGroupMemberCommitStatus =
+        new HashMap<TezVertexID, Boolean>();
 
     DAGSummaryData(TezDAGID dagId) {
       this.dagId = dagId;
@@ -356,7 +505,6 @@ public class RecoveryParser {
         case DAG_KILL_REQUEST:
           DAGKillRequestEvent killRequestEvent = new DAGKillRequestEvent();
           killRequestEvent.fromSummaryProtoStream(proto);
-          bufferedSummaryEvents.add(killRequestEvent);
           break;
         case DAG_COMMIT_STARTED:
           dagCommitCompleted = false;
@@ -375,24 +523,27 @@ public class RecoveryParser {
           if (vertexCommitStatus.containsKey(vertexFinishedEvent.getVertexID())) {
             vertexCommitStatus.put(
                 vertexFinishedEvent.getVertexID(), true);
-            bufferedSummaryEvents.add(vertexFinishedEvent);
           }
           break;
         case VERTEX_GROUP_COMMIT_STARTED:
           VertexGroupCommitStartedEvent vertexGroupCommitStartedEvent =
               new VertexGroupCommitStartedEvent();
           vertexGroupCommitStartedEvent.fromSummaryProtoStream(proto);
-          bufferedSummaryEvents.add(vertexGroupCommitStartedEvent);
           vertexGroupCommitStatus.put(
               vertexGroupCommitStartedEvent.getVertexGroupName(), false);
+          for (TezVertexID member : vertexGroupCommitStartedEvent.getVertexIds()) {
+            vertexGroupMemberCommitStatus.put(member, false);
+          }
           break;
         case VERTEX_GROUP_COMMIT_FINISHED:
           VertexGroupCommitFinishedEvent vertexGroupCommitFinishedEvent =
               new VertexGroupCommitFinishedEvent();
           vertexGroupCommitFinishedEvent.fromSummaryProtoStream(proto);
-          bufferedSummaryEvents.add(vertexGroupCommitFinishedEvent);
           vertexGroupCommitStatus.put(
               vertexGroupCommitFinishedEvent.getVertexGroupName(), true);
+          for (TezVertexID member : vertexGroupCommitFinishedEvent.getVertexIds()) {
+            vertexGroupMemberCommitStatus.put(member, true);
+          }
           break;
         default:
           String message = "Found invalid summary event that was not handled"
@@ -401,6 +552,37 @@ public class RecoveryParser {
       }
     }
 
+    // Check whether DAG is recoverable based on DAGSummaryData
+    //  1. Whether vertex is in the middle of committing
+    //  2. Whether vertex group is in the middle of committing
+    private void checkRecoverableSummary() {
+      if (!dagCommitCompleted) {
+        this.nonRecoverable = true;
+        this.reason = "DAG Commit was in progress, not recoverable"
+            + ", dagId=" + dagId;
+      }
+      if (!vertexCommitStatus.isEmpty()) {
+        for (Entry<TezVertexID, Boolean> entry : vertexCommitStatus.entrySet()) {
+          if (!(entry.getValue().booleanValue())) {
+            this.nonRecoverable = true;
+            this.reason = "Vertex Commit was in progress, not recoverable"
+                + ", dagId=" + dagId
+                + ", vertexId=" + entry.getKey();
+          }
+        }
+      }
+      if (!vertexGroupCommitStatus.isEmpty()) {
+        for (Entry<String, Boolean> entry : vertexGroupCommitStatus.entrySet()) {
+          if (!(entry.getValue().booleanValue())) {
+            this.nonRecoverable = true;
+            this.reason = "Vertex Group Commit was in progress, not recoverable"
+                + ", dagId=" + dagId
+                + ", vertexGroup=" + entry.getKey();
+          }
+        }
+      }
+    }
+
     @Override
     public String toString() {
       StringBuilder sb = new StringBuilder();
@@ -426,32 +608,6 @@ public class RecoveryParser {
     }
   }
 
-  private String isDAGRecoverable(DAGSummaryData data) {
-    if (!data.dagCommitCompleted) {
-      return "DAG Commit was in progress, not recoverable"
-          + ", dagId=" + data.dagId;
-    }
-    if (!data.vertexCommitStatus.isEmpty()) {
-      for (Entry<TezVertexID, Boolean> entry : data.vertexCommitStatus.entrySet()) {
-        if (!(entry.getValue().booleanValue())) {
-          return "Vertex Commit was in progress, not recoverable"
-              + ", dagId=" + data.dagId
-              + ", vertexId=" + entry.getKey();
-        }
-      }
-    }
-    if (!data.vertexGroupCommitStatus.isEmpty()) {
-      for (Entry<String, Boolean> entry : data.vertexGroupCommitStatus.entrySet()) {
-        if (!(entry.getValue().booleanValue())) {
-          return "Vertex Group Commit was in progress, not recoverable"
-              + ", dagId=" + data.dagId
-              + ", vertexGroup=" + entry.getKey();
-        }
-      }
-    }
-    return null;
-  }
-
   private List<Path> getSummaryFiles() throws IOException {
     List<Path> summaryFiles = new ArrayList<Path>();
     for (int i = 1; i < currentAttemptId; ++i) {
@@ -483,11 +639,22 @@ public class RecoveryParser {
     return recoveryFiles;
   }
 
-  public RecoveredDAGData parseRecoveryData() throws IOException {
+  /**
+   * 1. Read Summary Recovery file and build DAGSummaryData
+   *    Check whether it is recoverable based on the summary file (whether dag is 
+   *    in the middle of committing)
+   * 2. Read the non-Summary Recovery file and build DAGRecoveryData
+   *    Check whether it is recoverable based on both the summary file and non-summary file
+   *    (whether vertex has completed its committing, but its full non-summary recovery events are not seen)
+   * @return DAGRecoveryData
+   * @throws IOException
+   */
+  public DAGRecoveryData parseRecoveryData() throws IOException {
     int dagCounter = 0;
     Map<TezDAGID, DAGSummaryData> dagSummaryDataMap =
         new HashMap<TezDAGID, DAGSummaryData>();
     List<Path> summaryFiles = getSummaryFiles();
+    LOG.debug("SummaryFile size:" + summaryFiles.size());
     for (Path summaryFile : summaryFiles) {
       FileStatus summaryFileStatus = recoveryFS.getFileStatus(summaryFile);
       LOG.info("Parsing summary file"
@@ -559,33 +726,24 @@ public class RecoveryParser {
     LOG.info("Checking if DAG is in recoverable state"
         + ", dagId=" + lastInProgressDAGData.dagId);
 
-    final RecoveredDAGData recoveredDAGData = new RecoveredDAGData();
-    if (lastInProgressDAGData.completed) {
-      recoveredDAGData.isCompleted = true;
-      recoveredDAGData.dagState = lastInProgressDAGData.dagState;
-    }
-
-    String nonRecoverableReason = isDAGRecoverable(lastInProgressDAGData);
-    if (nonRecoverableReason != null) {
-      LOG.warn("Found last inProgress DAG but not recoverable: "
-          + lastInProgressDAGData);
-      recoveredDAGData.nonRecoverable = true;
-      recoveredDAGData.reason = nonRecoverableReason;
-    }
-
+    final DAGRecoveryData recoveredDAGData = new DAGRecoveryData(lastInProgressDAGData);
     List<Path> dagRecoveryFiles = getDAGRecoveryFiles(lastInProgressDAG);
     boolean skipAllOtherEvents = false;
     Path lastRecoveryFile = null;
+    // read the non summary events even when it is nonrecoverable. (Just read the DAGSubmittedEvent
+    // to create the DAGImpl)
     for (Path dagRecoveryFile : dagRecoveryFiles) {
       if (skipAllOtherEvents) {
         LOG.warn("Other recovery files will be skipped due to error in the previous recovery file"
             + lastRecoveryFile);
         break;
       }
+      FileStatus fileStatus = recoveryFS.getFileStatus(dagRecoveryFile);
       lastRecoveryFile = dagRecoveryFile;
       LOG.info("Trying to recover dag from recovery file"
           + ", dagId=" + lastInProgressDAG.toString()
-          + ", dagRecoveryFile=" + dagRecoveryFile);
+          + ", dagRecoveryFile=" + dagRecoveryFile
+          + ", len=" + fileStatus.getLen());
       FSDataInputStream dagRecoveryStream = recoveryFS.open(dagRecoveryFile, recoveryBufferSize);
       while (true) {
         HistoryEvent event;
@@ -606,14 +764,14 @@ public class RecoveryParser {
           // hit an error - skip reading other events
           break;
         }
+
         HistoryEventType eventType = event.getEventType();
+        LOG.info("Recovering from event"
+            + ", eventType=" + eventType
+            + ", event=" + event.toString());
         switch (eventType) {
           case DAG_SUBMITTED:
-          {
             DAGSubmittedEvent submittedEvent = (DAGSubmittedEvent) event;
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
             recoveredDAGData.recoveredDAG = dagAppMaster.createDAG(submittedEvent.getDAGPlan(),
                 lastInProgressDAG);
             recoveredDAGData.cumulativeAdditionalResources = submittedEvent
@@ -624,195 +782,110 @@ public class RecoveryParser {
               skipAllOtherEvents = true;
             }
             break;
-          }
           case DAG_INITIALIZED:
-          {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            recoveredDAGData.recoveredDAG.restoreFromEvent(event);
+            recoveredDAGData.dagInitedEvent = (DAGInitializedEvent)event;
             break;
-          }
           case DAG_STARTED:
-          {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            recoveredDAGData.recoveredDAG.restoreFromEvent(event);
+            recoveredDAGData.dagStartedEvent= (DAGStartedEvent)event;
             break;
-          }
+          case DAG_FINISHED:
+            recoveredDAGData.dagFinishedEvent = (DAGFinishedEvent)event;
+            skipAllOtherEvents = true;
+            break; 
           case DAG_COMMIT_STARTED:
-          {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            recoveredDAGData.recoveredDAG.restoreFromEvent(event);
-            break;
-          }
           case VERTEX_GROUP_COMMIT_STARTED:
+          case VERTEX_GROUP_COMMIT_FINISHED: 
+          case CONTAINER_LAUNCHED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            recoveredDAGData.recoveredDAG.restoreFromEvent(event);
-            break;
-          }
-          case VERTEX_GROUP_COMMIT_FINISHED:
-          {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            recoveredDAGData.recoveredDAG.restoreFromEvent(event);
+            // Nothing to do for now
             break;
           }
           case DAG_KILL_REQUEST:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            break;
-          }
-          case DAG_FINISHED:
-          {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            // If this is seen, nothing to recover
-            assert recoveredDAGData.recoveredDAG != null;
-            recoveredDAGData.recoveredDAG.restoreFromEvent(event);
-            recoveredDAGData.isCompleted = true;
-            recoveredDAGData.dagState =
-                ((DAGFinishedEvent) event).getState();
-            skipAllOtherEvents = true;
-            break;
-          }
-          case CONTAINER_LAUNCHED:
-          {
-            // Nothing to do for now
             break;
           }
           case VERTEX_INITIALIZED:
+
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            VertexInitializedEvent vEvent = (VertexInitializedEvent) event;
-            Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
-            v.restoreFromEvent(vEvent);
+            VertexInitializedEvent vertexInitEvent = (VertexInitializedEvent)event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.maybeCreateVertexRecoveryData(vertexInitEvent.getVertexID());
+            vertexRecoveryData.vertexInitedEvent = vertexInitEvent;
             break;
           }
-          case VERTEX_STARTED:
+          case VERTEX_CONFIGURE_DONE:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            VertexStartedEvent vEvent = (VertexStartedEvent) event;
-            Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
-            v.restoreFromEvent(vEvent);
+            VertexConfigurationDoneEvent reconfigureDoneEvent = (VertexConfigurationDoneEvent)event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.maybeCreateVertexRecoveryData(reconfigureDoneEvent.getVertexID());
+            vertexRecoveryData.vertexConfigurationDoneEvent = reconfigureDoneEvent;
             break;
           }
-          case VERTEX_PARALLELISM_UPDATED:
+          case VERTEX_STARTED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            VertexParallelismUpdatedEvent vEvent = (VertexParallelismUpdatedEvent) event;
-            Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
-            v.restoreFromEvent(vEvent);
+            VertexStartedEvent vertexStartedEvent = (VertexStartedEvent)event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(vertexStartedEvent.getVertexID());
+            Preconditions.checkArgument(vertexRecoveryData != null, "No VertexInitializedEvent before VertexStartedEvent");
+            vertexRecoveryData.vertexStartedEvent = vertexStartedEvent;
             break;
           }
           case VERTEX_COMMIT_STARTED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            VertexCommitStartedEvent vEvent = (VertexCommitStartedEvent) event;
-            Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
-            v.restoreFromEvent(vEvent);
             break;
           }
           case VERTEX_FINISHED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            VertexFinishedEvent vEvent = (VertexFinishedEvent) event;
-            Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
-            v.restoreFromEvent(vEvent);
+            VertexFinishedEvent vertexFinishedEvent = (VertexFinishedEvent)event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.maybeCreateVertexRecoveryData(vertexFinishedEvent.getVertexID());
+            vertexRecoveryData.vertexFinishedEvent = vertexFinishedEvent;
             break;
           }
           case TASK_STARTED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            TaskStartedEvent tEvent = (TaskStartedEvent) event;
-            Task task = recoveredDAGData.recoveredDAG.getVertex(
-                tEvent.getTaskID().getVertexID()).getTask(tEvent.getTaskID());
-            task.restoreFromEvent(tEvent);
+            TaskStartedEvent taskStartedEvent = (TaskStartedEvent) event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskStartedEvent.getTaskID().getVertexID());
+            Preconditions.checkArgument(vertexRecoveryData != null,
+                "Invalid TaskStartedEvent, its vertex does not exist:" + taskStartedEvent.getTaskID().getVertexID());
+            TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskStartedEvent.getTaskID());
+            taskRecoveryData.taskStartedEvent = taskStartedEvent;
             break;
           }
           case TASK_FINISHED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            TaskFinishedEvent tEvent = (TaskFinishedEvent) event;
-            Task task = recoveredDAGData.recoveredDAG.getVertex(
-                tEvent.getTaskID().getVertexID()).getTask(tEvent.getTaskID());
-            task.restoreFromEvent(tEvent);
+            TaskFinishedEvent taskFinishedEvent = (TaskFinishedEvent) event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(taskFinishedEvent.getTaskID().getVertexID());
+            Preconditions.checkArgument(vertexRecoveryData != null,
+                "Invalid TaskFinishedEvent, its vertex does not exist:" + taskFinishedEvent.getTaskID().getVertexID());
+            TaskRecoveryData taskRecoveryData = vertexRecoveryData.maybeCreateTaskRecoveryData(taskFinishedEvent.getTaskID());
+            taskRecoveryData.taskFinishedEvent = taskFinishedEvent;
             break;
           }
           case TASK_ATTEMPT_STARTED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            TaskAttemptStartedEvent tEvent = (TaskAttemptStartedEvent) event;
-            Task task =
-                recoveredDAGData.recoveredDAG.getVertex(
-                    tEvent.getTaskAttemptID().getTaskID().getVertexID())
-                        .getTask(tEvent.getTaskAttemptID().getTaskID());
-            task.restoreFromEvent(tEvent);
+            TaskAttemptStartedEvent taStartedEvent = (TaskAttemptStartedEvent)event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(
+                taStartedEvent.getTaskAttemptID().getTaskID().getVertexID());
+            Preconditions.checkArgument(vertexRecoveryData != null,
+                "Invalid TaskAttemptStartedEvent, its vertexId does not exist, taId=" + taStartedEvent.getTaskAttemptID());
+            TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap
+                .get(taStartedEvent.getTaskAttemptID().getTaskID());
+            Preconditions.checkArgument(taskRecoveryData != null,
+                "Invalid TaskAttemptStartedEvent, its taskId does not exist, taId=" + taStartedEvent.getTaskAttemptID());
+            TaskAttemptRecoveryData taRecoveryData = taskRecoveryData.maybeCreateTaskAttemptRecoveryData(taStartedEvent.getTaskAttemptID());
+            taRecoveryData.taStartedEvent = taStartedEvent;
             break;
           }
           case TASK_ATTEMPT_FINISHED:
           {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            TaskAttemptFinishedEvent tEvent = (TaskAttemptFinishedEvent) event;
-            Task task =
-                recoveredDAGData.recoveredDAG.getVertex(
-                    tEvent.getTaskAttemptID().getTaskID().getVertexID())
-                    .getTask(tEvent.getTaskAttemptID().getTaskID());
-            task.restoreFromEvent(tEvent);
-            break;
-          }
-          case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-          {
-            LOG.info("Recovering from event"
-                + ", eventType=" + eventType
-                + ", event=" + event.toString());
-            assert recoveredDAGData.recoveredDAG != null;
-            VertexRecoverableEventsGeneratedEvent vEvent =
-                (VertexRecoverableEventsGeneratedEvent) event;
-            Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
-            v.restoreFromEvent(vEvent);
+            TaskAttemptFinishedEvent taFinishedEvent = (TaskAttemptFinishedEvent)event;
+            VertexRecoveryData vertexRecoveryData = recoveredDAGData.vertexRecoveryDataMap.get(
+                taFinishedEvent.getTaskAttemptID().getTaskID().getVertexID());
+            Preconditions.checkArgument(vertexRecoveryData != null,
+                "Invalid TaskAttemtFinishedEvent, its vertexId does not exist, taId=" + taFinishedEvent.getTaskAttemptID());
+            TaskRecoveryData taskRecoveryData = vertexRecoveryData.taskRecoveryDataMap
+                .get(taFinishedEvent.getTaskAttemptID().getTaskID());
+            Preconditions.checkArgument(taskRecoveryData != null,
+                "Invalid TaskAttemptFinishedEvent, its taskId does not exist, taId=" + taFinishedEvent.getTaskAttemptID());
+            TaskAttemptRecoveryData taRecoveryData = taskRecoveryData.maybeCreateTaskAttemptRecoveryData(taFinishedEvent.getTaskAttemptID());
+            taRecoveryData.taFinishedEvent = taFinishedEvent;
             break;
           }
           default:
@@ -828,49 +901,185 @@ public class RecoveryParser {
       }
       dagRecoveryStream.close();
     }
+    recoveredDAGData.checkRecoverableNonSummary();
+    return recoveredDAGData;
+  }
 
-    if (!recoveredDAGData.isCompleted
-        && !recoveredDAGData.nonRecoverable) {
-      if (lastInProgressDAGData.bufferedSummaryEvents != null
-        && !lastInProgressDAGData.bufferedSummaryEvents.isEmpty()) {
-        for (HistoryEvent bufferedEvent : lastInProgressDAGData.bufferedSummaryEvents) {
-          assert recoveredDAGData.recoveredDAG != null;
-          switch (bufferedEvent.getEventType()) {
-            case VERTEX_GROUP_COMMIT_STARTED:
-              recoveredDAGData.recoveredDAG.restoreFromEvent(bufferedEvent);
-              break;
-            case VERTEX_GROUP_COMMIT_FINISHED:
-              recoveredDAGData.recoveredDAG.restoreFromEvent(bufferedEvent);
-              break;
-            case VERTEX_FINISHED:
-              VertexFinishedEvent vertexFinishedEvent =
-                  (VertexFinishedEvent) bufferedEvent;
-              Vertex vertex = recoveredDAGData.recoveredDAG.getVertex(
-                  vertexFinishedEvent.getVertexID());
-              if (vertex == null) {
-                recoveredDAGData.nonRecoverable = true;
-                recoveredDAGData.reason = "All state could not be recovered"
-                    + ", vertex completed but events not flushed"
-                    + ", vertexId=" + vertexFinishedEvent.getVertexID();
-              } else {
-                vertex.restoreFromEvent(vertexFinishedEvent);
-              }
-              break;
-            case DAG_KILL_REQUEST:
-              DAGKillRequestEvent killRequestEvent = (DAGKillRequestEvent)bufferedEvent;
-              recoveredDAGData.isSessionStopped = killRequestEvent.isSessionStopped();
-              recoveredDAGData.recoveredDAG.restoreFromEvent(bufferedEvent);
-              break;
-            default:
-              throw new RuntimeException("Invalid data found in buffered summary events"
-                  + ", unknown event type "
-                  + bufferedEvent.getEventType());
-          }
-        }
+  public static class VertexRecoveryData {
+
+    private VertexInitializedEvent vertexInitedEvent;
+    private VertexConfigurationDoneEvent vertexConfigurationDoneEvent;
+    private VertexStartedEvent vertexStartedEvent;
+    private VertexFinishedEvent vertexFinishedEvent;
+    private Map<TezTaskID, TaskRecoveryData> taskRecoveryDataMap =
+        new HashMap<TezTaskID, RecoveryParser.TaskRecoveryData>();
+    private boolean commited;
+
+    @VisibleForTesting
+    public VertexRecoveryData(VertexInitializedEvent vertexInitedEvent,
+        VertexConfigurationDoneEvent vertexReconfigureDoneEvent,
+        VertexStartedEvent vertexStartedEvent,
+        VertexFinishedEvent vertexFinishedEvent,
+        Map<TezTaskID, TaskRecoveryData> taskRecoveryDataMap, boolean commited) {
+      super();
+      this.vertexInitedEvent = vertexInitedEvent;
+      this.vertexConfigurationDoneEvent = vertexReconfigureDoneEvent;
+      this.vertexStartedEvent = vertexStartedEvent;
+      this.vertexFinishedEvent = vertexFinishedEvent;
+      this.taskRecoveryDataMap = taskRecoveryDataMap;
+      this.commited = commited;
+    }
+
+    public VertexRecoveryData(boolean committed) {
+      this.commited = committed;
+    }
+ 
+    public VertexInitializedEvent getVertexInitedEvent() {
+      return vertexInitedEvent;
+    }
+
+    public VertexStartedEvent getVertexStartedEvent() {
+      return vertexStartedEvent;
+    }
+
+    public VertexFinishedEvent getVertexFinishedEvent() {
+      return vertexFinishedEvent;
+    }
+
+    public VertexConfigurationDoneEvent getVertexConfigurationDoneEvent() {
+      return vertexConfigurationDoneEvent;
+    }
+
+    public boolean isReconfigureDone() {
+      return vertexConfigurationDoneEvent != null;
+    }
+
+    public boolean isVertexInited() {
+      return vertexInitedEvent != null;
+    }
+
+    public boolean shouldSkipInit() {
+      return vertexInitedEvent != null && vertexConfigurationDoneEvent != null;
+    }
+
+    public boolean isVertexStarted() {
+      return vertexStartedEvent != null;
+    }
+
+    public boolean isVertexSucceeded() {
+      if (vertexFinishedEvent == null) {
+        return false;
       }
+      return vertexFinishedEvent.getState().equals(VertexState.SUCCEEDED);
     }
 
-    return recoveredDAGData;
+    public boolean isVertexFinished() {
+      return vertexFinishedEvent != null;
+    }
+
+    public boolean isVertexCommitted() {
+      return this.commited;
+    }
+
+    public TaskRecoveryData getTaskRecoveryData(TezTaskID taskId) {
+      return taskRecoveryDataMap.get(taskId);
+    }
+
+    public TaskRecoveryData maybeCreateTaskRecoveryData(TezTaskID taskId) {
+      TaskRecoveryData taskRecoveryData = taskRecoveryDataMap.get(taskId);
+      if (taskRecoveryData == null) {
+        taskRecoveryData = new TaskRecoveryData();
+        taskRecoveryDataMap.put(taskId, taskRecoveryData);
+      }
+      return taskRecoveryData;
+    }
+
+    @Override
+    public String toString() {
+      StringBuilder builder = new StringBuilder();
+      builder.append("VertexInitedEvent=" + vertexInitedEvent);
+      builder.append("");
+      return builder.toString();
+    }
   }
 
+  public static class TaskRecoveryData {
+
+    private TaskStartedEvent taskStartedEvent;
+    private TaskFinishedEvent taskFinishedEvent;
+    private Map<TezTaskAttemptID, TaskAttemptRecoveryData> taRecoveryDataMap =
+        new HashMap<TezTaskAttemptID, RecoveryParser.TaskAttemptRecoveryData>();
+
+    public TaskRecoveryData() {
+
+    }
+
+    @VisibleForTesting
+    public TaskRecoveryData(TaskStartedEvent taskStartedEvent,
+        TaskFinishedEvent taskFinishedEvent,
+        Map<TezTaskAttemptID, TaskAttemptRecoveryData> taRecoveryDataMap) {
+      super();
+      this.taskStartedEvent = taskStartedEvent;
+      this.taskFinishedEvent = taskFinishedEvent;
+      this.taRecoveryDataMap = taRecoveryDataMap;
+    }
+
+    public TaskStartedEvent getTaskStartedEvent() {
+      return taskStartedEvent;
+    }
+
+    public TaskFinishedEvent getTaskFinishedEvent() {
+      return taskFinishedEvent;
+    }
+
+    public boolean isTaskStarted() {
+      return getTaskStartedEvent() != null;
+    }
+
+    public boolean isTaskAttemptSucceeded(TezTaskAttemptID taId) {
+      TaskAttemptRecoveryData taRecoveryData = taRecoveryDataMap.get(taId);
+      return taRecoveryData == null ? false : taRecoveryData.isTaskAttemptSucceeded();
+    }
+
+    public TaskAttemptRecoveryData maybeCreateTaskAttemptRecoveryData(TezTaskAttemptID taId) {
+      TaskAttemptRecoveryData taRecoveryData = taRecoveryDataMap.get(taId);
+      if (taRecoveryData == null) {
+        taRecoveryData = new TaskAttemptRecoveryData();
+        taRecoveryDataMap.put(taId, taRecoveryData);
+      }
+      return taRecoveryData;
+    }
+  }
+
+  public static class TaskAttemptRecoveryData {
+
+    private TaskAttemptStartedEvent taStartedEvent;
+    private TaskAttemptFinishedEvent taFinishedEvent;
+
+    public TaskAttemptRecoveryData() {
+
+    }
+
+    @VisibleForTesting
+    public TaskAttemptRecoveryData(TaskAttemptStartedEvent taStartedEvent,
+        TaskAttemptFinishedEvent taFinishedEvent) {
+      super();
+      this.taStartedEvent = taStartedEvent;
+      this.taFinishedEvent = taFinishedEvent;
+    }
+
+    public TaskAttemptStartedEvent getTaskAttemptStartedEvent() {
+      return taStartedEvent;
+    }
+
+    public TaskAttemptFinishedEvent getTaskAttemptFinishedEvent() {
+      return taFinishedEvent;
+    }
+
+    public boolean isTaskAttemptSucceeded() {
+      TaskAttemptFinishedEvent taFinishedEvent = getTaskAttemptFinishedEvent();
+      return taFinishedEvent == null ?
+          false : taFinishedEvent.getState() == TaskAttemptState.SUCCEEDED;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java
index 924222a..92bf3c4 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/TaskCommunicatorManager.java
@@ -36,7 +36,9 @@ import org.apache.tez.dag.api.UserPayload;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventType;
 import org.apache.tez.serviceplugins.api.ContainerEndReason;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStatusUpdate;
+import org.apache.tez.runtime.api.events.TaskAttemptFailedEvent;
 import org.apache.tez.runtime.api.events.TaskStatusUpdateEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
 import org.apache.tez.runtime.api.impl.EventType;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
 import org.slf4j.Logger;
@@ -57,11 +59,14 @@ import org.apache.hadoop.yarn.util.ConverterUtils;
 import org.apache.tez.dag.api.TaskHeartbeatRequest;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.app.dag.Task;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEvent;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptFailed;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventAttemptKilled;
 import org.apache.tez.dag.app.dag.event.TaskAttemptEventStartedRemotely;
+import org.apache.tez.dag.app.dag.event.TaskAttemptEventTezEventUpdate;
 import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.app.rm.container.AMContainerTask;
+import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.runtime.api.impl.TezEvent;
@@ -227,12 +232,14 @@ public class TaskCommunicatorManager extends AbstractService implements
       }
 
       long currTime = context.getClock().getTime();
-      List<TezEvent> otherEvents = new ArrayList<TezEvent>();
-      // route TASK_STATUS_UPDATE_EVENT directly to TaskAttempt and route other events
-      // (DATA_MOVEMENT_EVENT, TASK_ATTEMPT_COMPLETED_EVENT, TASK_ATTEMPT_FAILED_EVENT)
-      // to VertexImpl to ensure the events ordering
-      //  1. DataMovementEvent is logged as RecoveryEvent before TaskAttemptFinishedEvent
-      //  2. TaskStatusEvent is handled before TaskAttemptFinishedEvent
+      // taFinishedEvents - means the TaskAttemptFinishedEvent
+      // taGeneratedEvents - for recovery, means the events generated by this task attempt and is needed by its downstream vertices
+      // eventsForVertex - including all the taGeneratedEvents and other events such as INPUT_READ_ERROR_EVENT/INPUT_FAILED_EVENT
+      // taGeneratedEvents is routed both to TaskAttempt & Vertex. Route to Vertex is for performance consideration
+      // taFinishedEvents must be routed before taGeneratedEvents
+      List<TezEvent> taFinishedEvents = new ArrayList<TezEvent>();
+      List<TezEvent> taGeneratedEvents = new ArrayList<TezEvent>();
+      List<TezEvent> eventsForVertex = new ArrayList<TezEvent>();
       TaskAttemptEventStatusUpdate taskAttemptEvent = null;
       boolean readErrorReported = false;
       for (TezEvent tezEvent : ListUtils.emptyIfNull(inEvents)) {
@@ -244,21 +251,74 @@ public class TaskCommunicatorManager extends AbstractService implements
           // send TA_STATUS_UPDATE before TA_DONE/TA_FAILED/TA_KILLED otherwise Status may be missed
           taskAttemptEvent = new TaskAttemptEventStatusUpdate(taskAttemptID,
               (TaskStatusUpdateEvent) tezEvent.getEvent());
+        } else if (eventType == EventType.TASK_ATTEMPT_COMPLETED_EVENT
+           || eventType == EventType.TASK_ATTEMPT_FAILED_EVENT) {
+          taFinishedEvents.add(tezEvent);
         } else {
           if (eventType == EventType.INPUT_READ_ERROR_EVENT) {
             readErrorReported = true;
           }
-          otherEvents.add(tezEvent);
+          if (eventType == EventType.DATA_MOVEMENT_EVENT
+            || eventType == EventType.COMPOSITE_DATA_MOVEMENT_EVENT
+            || eventType == EventType.ROOT_INPUT_INITIALIZER_EVENT
+            || eventType == EventType.VERTEX_MANAGER_EVENT) {
+            taGeneratedEvents.add(tezEvent);
+          }
+          eventsForVertex.add(tezEvent);
         }
       }
       if (taskAttemptEvent != null) {
         taskAttemptEvent.setReadErrorReported(readErrorReported);
         context.getEventHandler().handle(taskAttemptEvent);
       }
-      if(!otherEvents.isEmpty()) {
+      // route taGeneratedEvents to TaskAttempt
+      if (!taGeneratedEvents.isEmpty()) {
+        context.getEventHandler().handle(new TaskAttemptEventTezEventUpdate(taskAttemptID, taGeneratedEvents));
+      }
+      // route events to TaskAttempt
+      Preconditions.checkArgument(taFinishedEvents.size() <= 1, "Multiple TaskAttemptFinishedEvent");
+      for (TezEvent e : taFinishedEvents) {
+        EventMetaData sourceMeta = e.getSourceInfo();
+        switch (e.getEventType()) {
+        case TASK_ATTEMPT_FAILED_EVENT:
+          TaskAttemptTerminationCause errCause = null;
+          switch (sourceMeta.getEventGenerator()) {
+          case INPUT:
+            errCause = TaskAttemptTerminationCause.INPUT_READ_ERROR;
+            break;
+          case PROCESSOR:
+            errCause = TaskAttemptTerminationCause.APPLICATION_ERROR;
+            break;
+          case OUTPUT:
+            errCause = TaskAttemptTerminationCause.OUTPUT_WRITE_ERROR;
+            break;
+          case SYSTEM:
+            errCause = TaskAttemptTerminationCause.FRAMEWORK_ERROR;
+            break;
+          default:
+            throw new TezUncheckedException("Unknown EventProducerConsumerType: " +
+                sourceMeta.getEventGenerator());
+          }
+          TaskAttemptFailedEvent taskFailedEvent =(TaskAttemptFailedEvent) e.getEvent();
+          context.getEventHandler().handle(
+               new TaskAttemptEventAttemptFailed(sourceMeta.getTaskAttemptID(),
+                   TaskAttemptEventType.TA_FAILED,
+                  "Error: " + taskFailedEvent.getDiagnostics(),
+                    errCause));
+          break;
+        case TASK_ATTEMPT_COMPLETED_EVENT:
+          context.getEventHandler().handle(
+              new TaskAttemptEvent(sourceMeta.getTaskAttemptID(), TaskAttemptEventType.TA_DONE));
+          break;
+        default:
+          throw new TezUncheckedException("Unhandled tez event type: "
+             + e.getEventType());
+        }
+      }
+      if (!eventsForVertex.isEmpty()) {
         TezVertexID vertexId = taskAttemptID.getTaskID().getVertexID();
         context.getEventHandler().handle(
-            new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(otherEvents)));
+            new VertexEventRouteEvent(vertexId, Collections.unmodifiableList(eventsForVertex)));
       }
       taskHeartbeatHandler.pinged(taskAttemptID);
       eventInfo = context
@@ -269,6 +329,7 @@ public class TaskCommunicatorManager extends AbstractService implements
     }
     return new TaskHeartbeatResponse(false, eventInfo.getEvents(), eventInfo.getNextFromEventId(), eventInfo.getNextPreRoutedFromEventId());
   }
+
   public void taskAlive(TezTaskAttemptID taskAttemptId) {
     taskHeartbeatHandler.pinged(taskAttemptId);
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
index 640359d..a01c623 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/DAG.java
@@ -34,7 +34,6 @@ import org.apache.tez.dag.api.client.StatusGetOpts;
 import org.apache.tez.dag.api.client.VertexStatusBuilder;
 import org.apache.tez.dag.api.records.DAGProtos.DAGPlan;
 import org.apache.tez.common.security.ACLManager;
-import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezVertexID;
 
@@ -90,8 +89,6 @@ public interface DAG {
   
   UserGroupInformation getDagUGI();
 
-  DAGState restoreFromEvent(HistoryEvent historyEvent);
-
   ACLManager getACLManager();
 
   Map<String, TezVertexID> getVertexNameIDMapping();

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
index a011b61..04f0e5b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Task.java
@@ -26,7 +26,6 @@ import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.TaskLocationHint;
 import org.apache.tez.dag.api.oldrecords.TaskReport;
 import org.apache.tez.dag.api.oldrecords.TaskState;
-import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.runtime.api.impl.TaskSpec;
@@ -65,8 +64,6 @@ public interface Task {
   
   public List<String> getDiagnostics();
 
-  TaskState restoreFromEvent(HistoryEvent historyEvent);
-
   public void registerTezEvent(TezEvent tezEvent);
   
   public TaskSpec getBaseTaskSpec();

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
index cbe72c1..ba09bd9 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskAttempt.java
@@ -28,7 +28,6 @@ import org.apache.tez.common.counters.DAGCounter;
 import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptReport;
 import org.apache.tez.dag.api.oldrecords.TaskAttemptState;
-import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezDAGID;
 import org.apache.tez.dag.records.TezTaskAttemptID;
@@ -136,7 +135,5 @@ public interface TaskAttempt {
    *  yet, returns 0.
    */
   long getFinishTime();
-  
-  TaskAttemptState restoreFromEvent(HistoryEvent event);
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
index 60f5a8f..9fc73a2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/Vertex.java
@@ -49,7 +49,6 @@ import org.apache.tez.dag.app.TaskAttemptEventInfo;
 import org.apache.tez.dag.app.dag.event.SpeculatorEvent;
 import org.apache.tez.dag.app.dag.impl.AMUserCodeException;
 import org.apache.tez.dag.app.dag.impl.Edge;
-import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
@@ -173,8 +172,6 @@ public interface Vertex extends Comparable<Vertex> {
   // internal apis
   AppContext getAppContext();
 
-  VertexState restoreFromEvent(HistoryEvent event);
-
   String getLogIdentifier();
 
   public void incrementFailedTaskAttemptCount();

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexState.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexState.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexState.java
index 1b7ac0f..ba9d1af 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexState.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/VertexState.java
@@ -27,6 +27,5 @@ public enum VertexState {
   KILLED,
   ERROR,
   TERMINATING,
-  RECOVERING,
   COMMITTING,
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventRecoverEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventRecoverEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventRecoverEvent.java
index 45e44f3..8e1edf0 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventRecoverEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/DAGEventRecoverEvent.java
@@ -18,37 +18,34 @@
 
 package org.apache.tez.dag.app.dag.event;
 
-import java.net.URL;
-import java.util.List;
-
+import org.apache.tez.dag.app.RecoveryParser.DAGRecoveryData;
 import org.apache.tez.dag.app.dag.DAGState;
 import org.apache.tez.dag.records.TezDAGID;
 
 public class DAGEventRecoverEvent extends DAGEvent {
 
   private final DAGState desiredState;
-  private final List<URL> additionalUrlsForClasspath;
+  private final DAGRecoveryData recoveredDagData;
 
-  public DAGEventRecoverEvent(TezDAGID dagId, DAGState desiredState,
-      List<URL> additionalUrlsForClasspath) {
+  public DAGEventRecoverEvent(TezDAGID dagId, DAGState desiredState, DAGRecoveryData recoveredDagData) {
     super(dagId, DAGEventType.DAG_RECOVER);
     this.desiredState = desiredState;
-    this.additionalUrlsForClasspath = additionalUrlsForClasspath;
+    this.recoveredDagData = recoveredDagData;
   }
   
-  public DAGEventRecoverEvent(TezDAGID dagId, List<URL> additionalUrlsForClasspath) {
-    this(dagId, null, additionalUrlsForClasspath);
+  public DAGEventRecoverEvent(TezDAGID dagId, DAGRecoveryData recoveredDagData) {
+    this(dagId, null, recoveredDagData);
   }
   
   public DAGState getDesiredState() {
     return desiredState;
   }
   
-  public List<URL> getAdditionalUrlsForClasspath() {
-    return this.additionalUrlsForClasspath;
-  }
-
   public boolean hasDesiredState() {
     return this.desiredState != null;
   }
+
+  public DAGRecoveryData getRecoveredDagData() {
+    return recoveredDagData;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/RecoveryEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/RecoveryEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/RecoveryEvent.java
new file mode 100644
index 0000000..cad3824
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/RecoveryEvent.java
@@ -0,0 +1,23 @@
+/**
+ * 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.dag.event;
+
+public interface RecoveryEvent {
+
+  public boolean isFromRecovery();
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
index 7ec8921..21c6b14 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptFailed.java
@@ -22,10 +22,11 @@ import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
 public class TaskAttemptEventAttemptFailed extends TaskAttemptEvent 
-  implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent {
+  implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent, RecoveryEvent {
 
   private final String diagnostics;
   private final TaskAttemptTerminationCause errorCause;
+  private boolean isFromRecovery = false;
 
   /* Accepted Types - FAILED, TIMED_OUT */
   public TaskAttemptEventAttemptFailed(TezTaskAttemptID id,
@@ -35,6 +36,14 @@ public class TaskAttemptEventAttemptFailed extends TaskAttemptEvent
     this.errorCause = errorCause;
   }
 
+  /* Accepted Types - FAILED, TIMED_OUT */
+  public TaskAttemptEventAttemptFailed(TezTaskAttemptID id,
+      TaskAttemptEventType type, String diagnostics, TaskAttemptTerminationCause errorCause,
+      boolean isFromRecovery) {
+    this(id, type, diagnostics, errorCause);
+    this.isFromRecovery = isFromRecovery;
+  }
+
   @Override
   public String getDiagnosticInfo() {
     return diagnostics;
@@ -45,4 +54,8 @@ public class TaskAttemptEventAttemptFailed extends TaskAttemptEvent
     return errorCause;
   }
 
+  @Override
+  public boolean isFromRecovery() {
+    return isFromRecovery;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
index 72e6b07..4642443 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventAttemptKilled.java
@@ -22,10 +22,12 @@ import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
 public class TaskAttemptEventAttemptKilled extends TaskAttemptEvent
-    implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent {
+    implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent, RecoveryEvent {
 
   private final String diagnostics;
   private final TaskAttemptTerminationCause errorCause;
+  private boolean fromRecovery;
+
   public TaskAttemptEventAttemptKilled(TezTaskAttemptID id,
                                        String diagnostics,
                                        TaskAttemptTerminationCause errorCause) {
@@ -34,6 +36,14 @@ public class TaskAttemptEventAttemptKilled extends TaskAttemptEvent
     this.errorCause = errorCause;
   }
 
+  public TaskAttemptEventAttemptKilled(TezTaskAttemptID id,
+      String diagnostics,
+      TaskAttemptTerminationCause errorCause,
+      boolean fromRecovery) {
+    this(id, diagnostics, errorCause);
+    this.fromRecovery = fromRecovery;
+  }
+  
   @Override
   public String getDiagnosticInfo() {
     return diagnostics;
@@ -44,4 +54,8 @@ public class TaskAttemptEventAttemptKilled extends TaskAttemptEvent
     return errorCause;
   }
 
+  @Override
+  public boolean isFromRecovery() {
+    return fromRecovery;
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventKillRequest.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventKillRequest.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventKillRequest.java
index a0dfe5d..96cf0e6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventKillRequest.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventKillRequest.java
@@ -21,10 +21,11 @@ import org.apache.tez.dag.records.TaskAttemptTerminationCause;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
 public class TaskAttemptEventKillRequest extends TaskAttemptEvent 
-  implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent {
+  implements DiagnosableEvent, TaskAttemptEventTerminationCauseEvent, RecoveryEvent {
 
   private final String message;
   private final TaskAttemptTerminationCause errorCause;
+  private boolean fromRecovery = false;
 
   public TaskAttemptEventKillRequest(TezTaskAttemptID id, String message, TaskAttemptTerminationCause err) {
     super(id, TaskAttemptEventType.TA_KILL_REQUEST);
@@ -32,6 +33,12 @@ public class TaskAttemptEventKillRequest extends TaskAttemptEvent
     this.errorCause = err;
   }
 
+  public TaskAttemptEventKillRequest(TezTaskAttemptID id, String message, TaskAttemptTerminationCause err,
+      boolean fromRecovery) {
+    this(id, message, err);
+    this.fromRecovery = fromRecovery;
+  }
+
   @Override
   public String getDiagnosticInfo() {
     return message;
@@ -42,4 +49,9 @@ public class TaskAttemptEventKillRequest extends TaskAttemptEvent
     return errorCause;
   }
 
+  @Override
+  public boolean isFromRecovery() {
+    return fromRecovery;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStartedRemotely.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStartedRemotely.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStartedRemotely.java
index 825a143..e700c6c 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStartedRemotely.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventStartedRemotely.java
@@ -24,11 +24,12 @@ import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.hadoop.yarn.api.records.ContainerId;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 
-public class TaskAttemptEventStartedRemotely extends TaskAttemptEvent {
+public class TaskAttemptEventStartedRemotely extends TaskAttemptEvent implements RecoveryEvent {
 
   private final ContainerId containerId;
   // TODO Can appAcls be handled elsewhere ?
   private final Map<ApplicationAccessType, String> applicationACLs;
+  private boolean fromRecovery = false;
 
   public TaskAttemptEventStartedRemotely(TezTaskAttemptID id, ContainerId containerId,
       Map<ApplicationAccessType, String> appAcls) {
@@ -37,6 +38,12 @@ public class TaskAttemptEventStartedRemotely extends TaskAttemptEvent {
     this.applicationACLs = appAcls;
   }
 
+  public TaskAttemptEventStartedRemotely(TezTaskAttemptID id, ContainerId containerId,
+      Map<ApplicationAccessType, String> appAcls, boolean fromRecovery) {
+    this(id, containerId, appAcls);
+    this.fromRecovery = fromRecovery;
+  }
+
   public ContainerId getContainerId() {
     return containerId;
   }
@@ -45,4 +52,9 @@ public class TaskAttemptEventStartedRemotely extends TaskAttemptEvent {
     return applicationACLs;
   }
 
+  @Override
+  public boolean isFromRecovery() {
+    return fromRecovery;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventTezEventUpdate.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventTezEventUpdate.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventTezEventUpdate.java
new file mode 100644
index 0000000..bef9248
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventTezEventUpdate.java
@@ -0,0 +1,37 @@
+/**
+ * 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.dag.event;
+
+import java.util.List;
+
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.api.impl.TezEvent;
+
+public class TaskAttemptEventTezEventUpdate extends TaskAttemptEvent {
+
+  private List<TezEvent> tezEvents;
+
+  public TaskAttemptEventTezEventUpdate(TezTaskAttemptID taId, List<TezEvent> tezEvents) {
+    super(taId, TaskAttemptEventType.TA_TEZ_EVENT_UPDATE);
+    this.tezEvents = tezEvents;
+  }
+
+  public List<TezEvent> getTezEvents() {
+    return tezEvents;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/28f30b0e/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
index 6ba69e3..dacb0c2 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/event/TaskAttemptEventType.java
@@ -29,6 +29,7 @@ public enum TaskAttemptEventType {
   //Producer: TaskAttemptListener | Vertex after routing events
   TA_STARTED_REMOTELY,
   TA_STATUS_UPDATE,
+  TA_TEZ_EVENT_UPDATE,  // for recovery
   TA_DONE,
   TA_FAILED,
   TA_KILLED, // Generated by TaskCommunicators
@@ -55,8 +56,5 @@ public enum TaskAttemptEventType {
   
   // Producer: consumer destination vertex
   TA_OUTPUT_FAILED,
-
-  // Recovery
-  TA_RECOVER,
   
 }