You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@tez.apache.org by je...@apache.org on 2014/09/18 21:49:46 UTC

[01/25] TEZ-850. Recovery unit tests. (Jeff Zhang via hitesh)

Repository: tez
Updated Branches:
  refs/heads/TEZ-8 d6589d3ac -> 625450cf1


http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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
index cd9a1e8..c5153b6 100644
--- 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
@@ -19,29 +19,28 @@
 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.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
 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.Container;
 import org.apache.hadoop.yarn.api.records.ContainerId;
-import org.apache.hadoop.yarn.api.records.LocalResource;
 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.Clock;
 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.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;
@@ -58,110 +57,532 @@ 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.rm.container.AMContainer;
 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.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 static final Log LOG = LogFactory.getLog(TestTaskImpl.class);
+  private TaskImpl task;
+  private DrainDispatcher dispatcher;
 
-  private int taskCounter = 0;
   private int taskAttemptCounter = 0;
 
-  private Configuration conf;
-  private TaskAttemptListener taskAttemptListener;
-  private TaskHeartbeatHandler taskHeartbeatHandler;
-  private Credentials credentials;
-  private Clock clock;
-  private ApplicationId appId;
-  private TezDAGID dagId;
-  private TezVertexID vertexId;
+  private Configuration conf = new Configuration();
+  private AppContext mockAppContext;
+  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 AppContext appContext;
-  private Resource taskResource;
-  private Map<String, LocalResource> localResources;
-  private Map<String, String> environment;
-  private String javaOpts;
-  private boolean leafVertex;
-  private ContainerContext containerContext;
-  private ContainerId mockContainerId;
-  private Container mockContainer;
-  private AMContainer mockAMContainer;
-  private NodeId mockNodeId;
-
-  private TaskImpl task;
-  private DrainDispatcher dispatcher;
+  private String vertexName = "v1";
+  private long taskScheduledTime = 100L;
+  private long taskStartTime = taskScheduledTime + 100L;
+  private long taskFinishTime = taskStartTime + 100L;
+  private TaskAttemptEventHandler taEventHandler =
+      new TaskAttemptEventHandler();
 
-  class TaskEventHandler implements EventHandler<TaskEvent> {
+  private class TaskEventHandler implements EventHandler<TaskEvent> {
     @Override
     public void handle(TaskEvent event) {
       task.handle(event);
     }
   }
 
-  class TaskAttemptEventHandler implements EventHandler<TaskAttemptEvent> {
+  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;
+    }
   }
 
-  @Before
-  public void setUp() {
-    conf = new Configuration();
-    taskAttemptListener = mock(TaskAttemptListener.class);
-    taskHeartbeatHandler = mock(TaskHeartbeatHandler.class);
-    credentials = new Credentials();
-    clock = new SystemClock();
-    appId = ApplicationId.newInstance(System.currentTimeMillis(), 1);
-    dagId = TezDAGID.getInstance(appId, 1);
-    vertexId = TezVertexID.getInstance(dagId, 1);
-    vertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
-    when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
-    appContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
-    mockContainerId = mock(ContainerId.class);
-    mockContainer = mock(Container.class);
-    mockAMContainer = mock(AMContainer.class);
-    mockNodeId = mock(NodeId.class);
-    when(mockContainer.getId()).thenReturn(mockContainerId);
-    when(mockContainer.getNodeId()).thenReturn(mockNodeId);
-    when(mockAMContainer.getContainer()).thenReturn(mockContainer);
-    when(appContext.getAllContainers().get(mockContainerId)).thenReturn(
-        mockAMContainer);
-    when(appContext.getCurrentDAG().getVertex(any(TezVertexID.class)))
-        .thenReturn(vertex);
-    when(vertex.getProcessorDescriptor().getClassName()).thenReturn("");
+  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 {
 
-    taskResource = Resource.newInstance(1024, 1);
-    localResources = new HashMap<String, LocalResource>();
-    environment = new HashMap<String, String>();
-    javaOpts = "";
-    leafVertex = false;
-    containerContext =
-        new ContainerContext(localResources, credentials, environment, javaOpts);
+    }
+
+  }
 
+  @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,
-        new TaskAttemptEventHandler());
+    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);
+
     task =
-        new TaskImpl(vertexId, 1, dispatcher.getEventHandler(), conf,
-            taskAttemptListener, clock, taskHeartbeatHandler, appContext,
-            leafVertex, taskResource, containerContext);
+        new TaskImpl(vertexId, 0, dispatcher.getEventHandler(),
+            new Configuration(), mock(TaskAttemptListener.class),
+            new SystemClock(), mock(TaskHeartbeatHandler.class),
+            mockAppContext, false, Resource.newInstance(1, 1),
+            mock(ContainerContext.class));
+
+    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.finishedAttempts);
+    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.finishedAttempts);
+    assertEquals(taskScheduledTime, task.scheduledTime);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(TaskAttemptStateInternal.NEW,
+        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+    assertEquals(1, task.numberUncompletedAttempts);
+  }
+
+  /**
+   * New -> RecoverTransition
+   */
+  @Test
+  public void testRecovery_New() {
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.NEW, task.getInternalState());
+  }
+
+  /**
+   * -> restoreFromTaskFinishEvent ( no TaskStartEvent )
+   */
+  @Test
+  public void testRecovery_NoStartEvent() {
+    try {
+      task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(), vertexName,
+          taskStartTime, taskFinishTime, null, TaskState.SUCCEEDED, "",
+          new TezCounters()));
+      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"));
+    }
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> RecoverTransition
+   */
+  @Test
+  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.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * RecoverTranstion
+   */
+  @Test
+  public void testRecovery_OneTAStarted() {
+    restoreFromTaskStartEvent();
+    TezTaskAttemptID taId = getNewTaskAttemptID(task.getTaskId());
+    restoreFromFirstTaskAttemptStartEvent(taId);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.RUNNING, task.getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(0, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  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, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
+    assertEquals(TaskStateInternal.SUCCEEDED, task.getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (FAILED) -> RecoverTransition
+   */
+  @Test
+  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, "",
+            new TezCounters()));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(1, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    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.finishedAttempts);
+    assertEquals(1, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (KILLED) -> RecoverTransition
+   */
+  @Test
+  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, "",
+            new TezCounters()));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    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.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) ->
+   * restoreFromTaskFinishedEvent -> RecoverTransition
+   */
+  @Test
+  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, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+
+    recoveredState =
+        task.restoreFromEvent(new TaskFinishedEvent(task.getTaskId(),
+            vertexName, taskStartTime, taskFinishTime, taId,
+            TaskState.SUCCEEDED, "", new TezCounters()));
+    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.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    assertEquals(taId, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  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, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    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.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  /**
+   * restoreFromTaskStartedEvent -> restoreFromTaskAttemptStartedEvent ->
+   * restoreFromTaskAttemptFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  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, "",
+            new TezCounters()));
+    assertEquals(TaskState.SUCCEEDED, recoveredState);
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    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.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+  }
+
+  @Test
+  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
+  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
+  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
+  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.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
+
+    long taFinishTime = taStartTime + 100L;
+    recoveredState =
+        task.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            taStartTime, taFinishTime, TaskAttemptState.KILLED, "",
+            new TezCounters()));
+    assertEquals(TaskState.RUNNING, recoveredState);
+    assertEquals(TaskAttemptStateInternal.NEW,
+        ((TaskAttemptImpl) task.getAttempt(taId)).getInternalState());
+    assertEquals(1, task.getAttempts().size());
+    assertEquals(1, task.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(0, task.numberUncompletedAttempts);
+    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.finishedAttempts);
+    assertEquals(0, task.failedAttempts);
+    assertEquals(1, task.numberUncompletedAttempts);
+    assertEquals(null, task.successfulAttempt);
   }
 
   /**
@@ -170,24 +591,23 @@ public class TestTaskRecovery {
    * schedule a new task attempt.
    */
   @Test
-  public void testTaskRecovery1() {
-    TezTaskID lastTaskId = getNewTaskID();
-    TezTaskID taskId = getNewTaskID();
+  public void testTaskRecovery_MultipleAttempts1() {
     int maxFailedAttempts =
         conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
             TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+    restoreFromTaskStartEvent();
+
     for (int i = 0; i < maxFailedAttempts; ++i) {
-      TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
-      task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
-          mockContainerId, mockNodeId, "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
-          TaskAttemptState.KILLED, "", null));
+      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));
     }
     assertEquals(maxFailedAttempts, task.getAttempts().size());
     assertEquals(0, task.failedAttempts);
 
-    task.handle(new TaskEventRecoverTask(lastTaskId));
+    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());
@@ -201,24 +621,23 @@ public class TestTaskRecovery {
    * failed_attempt is exceeded.
    */
   @Test
-  public void testTaskRecovery2() {
-    TezTaskID lastTaskId = getNewTaskID();
-    TezTaskID taskId = getNewTaskID();
+  public void testTaskRecovery_MultipleAttempts2() {
     int maxFailedAttempts =
         conf.getInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS,
             TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS_DEFAULT);
-    task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+    restoreFromTaskStartEvent();
+
     for (int i = 0; i < maxFailedAttempts; ++i) {
-      TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
-      task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
-          mockContainerId, mockNodeId, "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
-          TaskAttemptState.FAILED, "", null));
+      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));
     }
     assertEquals(maxFailedAttempts, task.getAttempts().size());
     assertEquals(maxFailedAttempts, task.failedAttempts);
 
-    task.handle(new TaskEventRecoverTask(lastTaskId));
+    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());
@@ -232,34 +651,34 @@ public class TestTaskRecovery {
    * state and new task attempt is scheduled.
    */
   @Test
-  public void testTaskRecovery3() throws InterruptedException {
-    TezTaskID lastTaskId = getNewTaskID();
-    TezTaskID taskId = getNewTaskID();
+  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);
-    task.restoreFromEvent(new TaskStartedEvent(taskId, "v1", 0, 0));
+    restoreFromTaskStartEvent();
+
     for (int i = 0; i < maxFailedAttempts - 1; ++i) {
-      TezTaskAttemptID attemptId = getNewTaskAttemptID(lastTaskId);
-      task.restoreFromEvent(new TaskAttemptStartedEvent(attemptId, "v1", 0,
-          mockContainerId, mockNodeId, "", ""));
-      task.restoreFromEvent(new TaskAttemptFinishedEvent(attemptId, "v1", 0, 0,
-          TaskAttemptState.FAILED, "", null));
+      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));
     }
     assertEquals(maxFailedAttempts - 1, task.getAttempts().size());
     assertEquals(maxFailedAttempts - 1, task.failedAttempts);
 
-    TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(lastTaskId);
+    TezTaskAttemptID newTaskAttemptId = getNewTaskAttemptID(task.getTaskId());
     TaskState recoveredState =
         task.restoreFromEvent(new TaskAttemptStartedEvent(newTaskAttemptId,
-            "v1", 0, mockContainerId, mockNodeId, "", ""));
+            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(lastTaskId));
+    task.handle(new TaskEventRecoverTask(task.getTaskId()));
     // wait until task attempt receive the Recover event from task
     dispatcher.await();
 
@@ -273,12 +692,8 @@ public class TestTaskRecovery {
     assertEquals(maxFailedAttempts + 1, task.getAttempts().size());
   }
 
-  private TezTaskID getNewTaskID() {
-    TezTaskID taskID = TezTaskID.getInstance(vertexId, ++taskCounter);
-    return taskID;
-  }
-
   private TezTaskAttemptID getNewTaskAttemptID(TezTaskID taskId) {
     return TezTaskAttemptID.getInstance(taskId, taskAttemptCounter++);
   }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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
new file mode 100644
index 0000000..e2f189c
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexRecovery.java
@@ -0,0 +1,860 @@
+/**
+ * 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.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.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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.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.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.TezEntityDescriptorProto;
+import org.apache.tez.dag.api.records.DAGProtos.VertexPlan;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.VertexState;
+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.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.VertexEventRecoverVertex;
+import org.apache.tez.dag.app.dag.event.VertexEventType;
+import org.apache.tez.dag.app.dag.impl.TestVertexImpl.CountingOutputCommitter;
+import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+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.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.Before;
+import org.junit.Test;
+
+import com.google.common.collect.Lists;
+
+public class TestVertexRecovery {
+
+  private static final Log LOG = LogFactory.getLog(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;
+  }
+
+  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 VertexEventHanlder vertexEventHandler;
+  private TaskEventHandler taskEventHandler;
+
+  @Before
+  public void setUp() throws IOException {
+
+    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(TaskAttemptListener.class),
+            new Credentials(), new SystemClock(), user,
+            mock(TaskHeartbeatHandler.class), mockAppContext);
+    when(mockAppContext.getCurrentDAG()).thenReturn(dag);
+
+    dag.handle(new DAGEvent(dagId, DAGEventType.DAG_INIT));
+    LOG.info("finish setUp");
+  }
+
+  /**
+   * vertex1(New) -> StartRecoveryTransition(SUCCEEDED)
+   */
+  @Test
+  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());
+
+  }
+
+  /**
+   * vertex1(New) -> StartRecoveryTransition(FAILED)
+   */
+  @Test
+  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
+  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
+  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
+  public void testRecovery_New_Desired_RUNNING() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    VertexState recoveredState =
+        vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+            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
+  public void testRecovery_Inited_Desired_RUNNING() {
+    VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
+    restoreFromInitializedEvent(vertex1);
+
+    VertexState recoveredState =
+        vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+            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
+  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 VertexDataMovementEventsGeneratedEvent(
+            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
+  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", initRequestedTime, initedTime, startRequestedTime,
+            startTime, finishTime, VertexState.SUCCEEDED, "",
+            new TezCounters(), new VertexStats()));
+    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 vertex3 (New) -> RecoverTransition
+   */
+  @Test
+  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 VertexDataMovementEventsGeneratedEvent(
+            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) -> restoreFromInitialized -> StartRecoveryTransition<br>
+   * vertex2 (New) -> restoreFromInitialized -> StartRecoveryTransition<br>
+   * vertex3 (New) -> restoreFromVertexInitedEvent -> RecoverTransition<br>
+   */
+  @Test
+  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 VertexDataMovementEventsGeneratedEvent(
+            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
+  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 VertexDataMovementEventsGeneratedEvent(
+            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
+  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", initRequestedTime, initedTime, initRequestedTime + 300L,
+        initRequestedTime + 400L, initRequestedTime + 500L,
+        VertexState.SUCCEEDED, "", new TezCounters(), new VertexStats()));
+    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 VertexDataMovementEventsGeneratedEvent(
+            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);
+  }
+}


[16/25] git commit: TEZ-1580. Change TestOrderedWordCount to optionally use MR configs. (hitesh)

Posted by je...@apache.org.
TEZ-1580. Change TestOrderedWordCount to optionally use MR configs. (hitesh)


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

Branch: refs/heads/TEZ-8
Commit: 9dd0cb4d8e933d5f57b3d9ae532e7167978aed68
Parents: 5e5683a
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 14:25:17 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 14:25:17 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../examples/TestOrderedWordCount.java          | 61 +++++++++++++++-----
 2 files changed, 49 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/9dd0cb4d/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 73a3671..f71c2e2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -16,6 +16,7 @@ ALL CHANGES:
   TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts.
   TEZ-1578. Remove TeraSort from Tez codebase.
   TEZ-1569. Add tests for preemption
+  TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/9dd0cb4d/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
index 2c5db10..a36d1d2 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/TestOrderedWordCount.java
@@ -149,7 +149,9 @@ public class TestOrderedWordCount extends Configured implements Tool {
   public DAG createDAG(FileSystem fs, Configuration conf,
       Map<String, LocalResource> commonLocalResources, Path stagingDir,
       int dagIndex, String inputPath, String outputPath,
-      boolean generateSplitsInClient) throws Exception {
+      boolean generateSplitsInClient,
+      boolean useMRSettings,
+      int intermediateNumReduceTasks) throws Exception {
 
     Configuration mapStageConf = new JobConf(conf);
     mapStageConf.set(MRJobConfig.MAP_CLASS_ATTR,
@@ -196,32 +198,62 @@ public class TestOrderedWordCount extends Configured implements Tool {
       dsd = MRInputLegacy.createConfigBuilder(mapStageConf, TextInputFormat.class, inputPath).build();
     }
 
-    Vertex mapVertex = Vertex.create("initialmap", ProcessorDescriptor.create(
-        MapProcessor.class.getName()).setUserPayload(
-        TezUtils.createUserPayloadFromConf(mapStageConf))
-        .setHistoryText(mapStageHistoryText)).addTaskLocalFiles(commonLocalResources);
-    mapVertex.addDataSource("MRInput", dsd);
+    Vertex mapVertex;
+    ProcessorDescriptor mapProcessorDescriptor =
+        ProcessorDescriptor.create(MapProcessor.class.getName())
+            .setUserPayload(
+                TezUtils.createUserPayloadFromConf(mapStageConf))
+            .setHistoryText(mapStageHistoryText);
+    if (!useMRSettings) {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor);
+    } else {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor, -1,
+          MRHelpers.getResourceForMRMapper(mapStageConf));
+      mapVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
+    }
+    mapVertex.addTaskLocalFiles(commonLocalResources)
+        .addDataSource("MRInput", dsd);
     vertices.add(mapVertex);
 
     ByteArrayOutputStream iROutputStream = new ByteArrayOutputStream(4096);
     iReduceStageConf.writeXml(iROutputStream);
     String iReduceStageHistoryText = new String(iROutputStream.toByteArray(), "UTF-8");
-    Vertex ivertex = Vertex.create("intermediate_reducer", ProcessorDescriptor.create(
+
+    ProcessorDescriptor iReduceProcessorDescriptor = ProcessorDescriptor.create(
         ReduceProcessor.class.getName())
         .setUserPayload(TezUtils.createUserPayloadFromConf(iReduceStageConf))
-        .setHistoryText(iReduceStageHistoryText), 2);
-    ivertex.addTaskLocalFiles(commonLocalResources);
-    vertices.add(ivertex);
+        .setHistoryText(iReduceStageHistoryText);
+
+    Vertex intermediateVertex;
+    if (!useMRSettings) {
+      intermediateVertex = Vertex.create("intermediate_reducer", iReduceProcessorDescriptor,
+          intermediateNumReduceTasks);
+    } else {
+      intermediateVertex = Vertex.create("intermediate_reducer", iReduceProcessorDescriptor,
+          intermediateNumReduceTasks, MRHelpers.getResourceForMRReducer(iReduceStageConf));
+      intermediateVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(iReduceStageConf));
+    }
+    intermediateVertex.addTaskLocalFiles(commonLocalResources);
+    vertices.add(intermediateVertex);
 
     ByteArrayOutputStream finalReduceOutputStream = new ByteArrayOutputStream(4096);
     finalReduceConf.writeXml(finalReduceOutputStream);
     String finalReduceStageHistoryText = new String(finalReduceOutputStream.toByteArray(), "UTF-8");
     UserPayload finalReducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
-    Vertex finalReduceVertex = Vertex.create("finalreduce",
+    Vertex finalReduceVertex;
+
+    ProcessorDescriptor finalReduceProcessorDescriptor =
         ProcessorDescriptor.create(
             ReduceProcessor.class.getName())
             .setUserPayload(finalReducePayload)
-            .setHistoryText(finalReduceStageHistoryText), 1);
+            .setHistoryText(finalReduceStageHistoryText);
+    if (!useMRSettings) {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1);
+    } else {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1,
+          MRHelpers.getResourceForMRReducer(finalReduceConf));
+      finalReduceVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
+    }
     finalReduceVertex.addTaskLocalFiles(commonLocalResources);
     finalReduceVertex.addDataSink("MROutput",
         MROutputLegacy.createConfigBuilder(finalReduceConf, TextOutputFormat.class, outputPath)
@@ -283,6 +315,9 @@ public class TestOrderedWordCount extends Configured implements Tool {
         * 1000;
 
     boolean retainStagingDir = conf.getBoolean("RETAIN_STAGING_DIR", false);
+    boolean useMRSettings = conf.getBoolean("USE_MR_CONFIGS", true);
+    // TODO needs to use auto reduce parallelism
+    int intermediateNumReduceTasks = conf.getInt("IREDUCE_NUM_TASKS", 2);
 
     if (((otherArgs.length%2) != 0)
         || (!useTezSession && otherArgs.length != 2)) {
@@ -371,7 +406,7 @@ public class TestOrderedWordCount extends Configured implements Tool {
         
         DAG dag = instance.createDAG(fs, conf, localResources,
             stagingDir, dagIndex, inputPath, outputPath,
-            generateSplitsInClient);
+            generateSplitsInClient, useMRSettings, intermediateNumReduceTasks);
 
         boolean doPreWarm = dagIndex == 1 && useTezSession
             && conf.getBoolean("PRE_WARM_SESSION", true);


[03/25] git commit: Revert "TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput() (Rajesh Balamohan)"

Posted by je...@apache.org.
Revert "TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput() (Rajesh Balamohan)"

This reverts commit de008b54a208881321d15addfe35b8094720be25.


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

Branch: refs/heads/TEZ-8
Commit: 80b5795b13b7483d876b741245708a56794cd771
Parents: f65e65a
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Thu Sep 11 03:07:16 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Sep 12 03:54:46 2014 +0530

----------------------------------------------------------------------
 .../dag/impl/ImmediateStartVertexManager.java   |  92 +----
 .../app/dag/impl/RootInputVertexManager.java    |  27 +-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    | 364 ++-----------------
 3 files changed, 66 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/80b5795b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
index ac2b851..b202d70 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
@@ -18,11 +18,9 @@
 
 package org.apache.tez.dag.app.dag.impl;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.tez.dag.api.EdgeProperty;
+import java.util.List;
+import java.util.Map;
+
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.VertexManagerPlugin;
 import org.apache.tez.dag.api.VertexManagerPluginContext;
@@ -30,101 +28,29 @@ import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
 import org.apache.tez.runtime.api.Event;
 import org.apache.tez.runtime.api.events.VertexManagerEvent;
 
-import java.util.List;
-import java.util.Map;
+import com.google.common.collect.Lists;
 
 /**
  * Starts all tasks immediately on vertex start
  */
 public class ImmediateStartVertexManager extends VertexManagerPlugin {
 
-  private static final Log LOG = LogFactory.getLog(ImmediateStartVertexManager.class);
-
-  private final Map<String, SourceVertexInfo> srcVertexInfo = Maps.newHashMap();
-  private int managedTasks;
-  private boolean tasksScheduled = false;
-
-  class SourceVertexInfo {
-    EdgeProperty edgeProperty;
-    int numFinishedTasks;
-
-    SourceVertexInfo(EdgeProperty edgeProperty) {
-      this.edgeProperty = edgeProperty;
-    }
-  }
-
   public ImmediateStartVertexManager(VertexManagerPluginContext context) {
     super(context);
   }
 
   @Override
   public void onVertexStarted(Map<String, List<Integer>> completions) {
-    managedTasks = getContext().getVertexNumTasks(getContext().getVertexName());
-    Map<String, EdgeProperty> edges = getContext().getInputVertexEdgeProperties();
-    for (Map.Entry<String, EdgeProperty> entry : edges.entrySet()) {
-      String srcVertex = entry.getKey();
-      EdgeProperty edgeProp = entry.getValue();
-      srcVertexInfo.put(srcVertex, new SourceVertexInfo(edgeProp));
-    }
-
-    //handle completions
-    for (Map.Entry<String, List<Integer>> entry : completions.entrySet()) {
-      for (Integer task : entry.getValue()) {
-        handleSourceTaskFinished(entry.getKey(), task);
-      }
-    }
-    scheduleTasks();
-  }
-
-  private void handleSourceTaskFinished(String vertex, Integer taskId) {
-    SourceVertexInfo srcInfo = srcVertexInfo.get(vertex);
-    //Not mandatory to check for duplicate completions here
-    srcInfo.numFinishedTasks++;
-  }
-
-  private void scheduleTasks() {
-    if (!canScheduleTasks()) {
-      return;
-    }
-
-    List<TaskWithLocationHint> tasksToStart = Lists.newArrayListWithCapacity(managedTasks);
-    for (int i = 0; i < managedTasks; ++i) {
-      tasksToStart.add(new TaskWithLocationHint(new Integer(i), null));
-    }
-
-    if (!tasksToStart.isEmpty()) {
-      LOG.info("Starting " + tasksToStart.size() + " in " + getContext().getVertexName());
-      getContext().scheduleVertexTasks(tasksToStart);
+    int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
+    List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
+    for (int i=0; i<numTasks; ++i) {
+      scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
     }
-    tasksScheduled = true;
-  }
-
-  private boolean canScheduleTasks() {
-    //Check if at least 1 task is finished from each source vertex (in case of broadcast &
-    // one-to-one or custom)
-    for (Map.Entry<String, SourceVertexInfo> entry : srcVertexInfo.entrySet()) {
-      SourceVertexInfo srcVertexInfo = entry.getValue();
-      switch(srcVertexInfo.edgeProperty.getDataMovementType()) {
-      case ONE_TO_ONE:
-      case BROADCAST:
-      case CUSTOM:
-        if (srcVertexInfo.numFinishedTasks == 0) {
-          //do not schedule tasks until a task from source task is complete
-          return false;
-        }
-      default:
-        break;
-      }
-    }
-    return true;
+    getContext().scheduleVertexTasks(scheduledTasks);
   }
 
   @Override
   public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
-    handleSourceTaskFinished(srcVertexName, attemptId);
-    if (!tasksScheduled) {
-      scheduleTasks();
-    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/80b5795b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
index e850286..e6ffdc5 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
@@ -18,23 +18,27 @@
 
 package org.apache.tez.dag.app.dag.impl;
 
+import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.VertexManagerPlugin;
 import org.apache.tez.dag.api.VertexManagerPluginContext;
+import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
 import org.apache.tez.runtime.api.Event;
 import org.apache.tez.runtime.api.InputSpecUpdate;
 import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
 import org.apache.tez.runtime.api.events.InputDataInformationEvent;
 import org.apache.tez.runtime.api.events.InputUpdatePayloadEvent;
+import org.apache.tez.runtime.api.events.VertexManagerEvent;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
-public class RootInputVertexManager extends ImmediateStartVertexManager {
+public class RootInputVertexManager extends VertexManagerPlugin {
 
   private String configuredInputName;
 
@@ -42,6 +46,27 @@ public class RootInputVertexManager extends ImmediateStartVertexManager {
     super(context);
   }
 
+  @Override
+  public void initialize() {
+  }
+
+  @Override
+  public void onVertexStarted(Map<String, List<Integer>> completions) {
+    int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
+    List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
+    for (int i=0; i<numTasks; ++i) {
+      scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
+    }
+    getContext().scheduleVertexTasks(scheduledTasks);
+  }
+
+  @Override
+  public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
+  }
+
+  @Override
+  public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) {
+  }
 
   @Override
   public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor,

http://git-wip-us.apache.org/repos/asf/tez/blob/80b5795b/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 04e2219..d894928 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
@@ -20,7 +20,6 @@ package org.apache.tez.dag.app.dag.impl;
 
 import java.nio.ByteBuffer;
 
-import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Mockito.doReturn;
@@ -1880,39 +1879,39 @@ public class TestVertexImpl {
     Assert.assertEquals(2, v3.getOutputVerticesCount());
     Assert.assertEquals(2, v3.getOutputVerticesCount());
 
-    assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
+    Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
         .getSourceVertexName())
         || "vertex2".equals(v3.getInputSpecList(0).get(0)
-        .getSourceVertexName()));
-    assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
+            .getSourceVertexName()));
+    Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
         .getSourceVertexName())
         || "vertex2".equals(v3.getInputSpecList(0).get(1)
-        .getSourceVertexName()));
-    assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
+            .getSourceVertexName()));
+    Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
         .getInputDescriptor().getClassName())
         || "i3_v2".equals(v3.getInputSpecList(0).get(0)
-        .getInputDescriptor().getClassName()));
-    assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
+            .getInputDescriptor().getClassName()));
+    Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
         .getInputDescriptor().getClassName())
         || "i3_v2".equals(v3.getInputSpecList(0).get(1)
-        .getInputDescriptor().getClassName()));
+            .getInputDescriptor().getClassName()));
 
-    assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
+    Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
         .getDestinationVertexName())
         || "vertex5".equals(v3.getOutputSpecList(0).get(0)
-        .getDestinationVertexName()));
-    assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
+            .getDestinationVertexName()));
+    Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
         .getDestinationVertexName())
         || "vertex5".equals(v3.getOutputSpecList(0).get(1)
-        .getDestinationVertexName()));
-    assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
+            .getDestinationVertexName()));
+    Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
         .getOutputDescriptor().getClassName())
         || "o3_v5".equals(v3.getOutputSpecList(0).get(0)
-        .getOutputDescriptor().getClassName()));
-    assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
+            .getOutputDescriptor().getClassName()));
+    Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
         .getOutputDescriptor().getClassName())
         || "o3_v5".equals(v3.getOutputSpecList(0).get(1)
-        .getOutputDescriptor().getClassName()));
+            .getOutputDescriptor().getClassName()));
   }
 
   @Test(timeout = 5000)
@@ -1941,13 +1940,13 @@ public class TestVertexImpl {
     Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
         Collections.singletonMap(
        v1.getName(), mockEdgeManagerDescriptor);
-    assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
-    assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
+    Assert.assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
+    Assert.assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
         EdgeManagerForTest);
     Assert.assertEquals(1, v3.getTotalTasks());
     Assert.assertEquals(1, tasks.size());
     // the last one is removed
-    assertTrue(tasks.keySet().iterator().next().equals(firstTask));
+    Assert.assertTrue(tasks.keySet().iterator().next().equals(firstTask));
 
   }
   
@@ -1994,7 +1993,7 @@ public class TestVertexImpl {
     Edge edge = edges.get("e4");
     EdgeManagerPlugin em = edge.getEdgeManager();
     EdgeManagerForTest originalEm = (EdgeManagerForTest) em;
-    assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
+    Assert.assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
         .getUserPayload().deepCopyAsArray()));
 
     UserPayload userPayload = UserPayload.create(ByteBuffer.wrap(new String("foo").getBytes()));
@@ -2008,7 +2007,7 @@ public class TestVertexImpl {
 
     Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
         Collections.singletonMap(v3.getName(), edgeManagerDescriptor);
-    assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
+    Assert.assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
         edgeManagerDescriptors, null)); // Must decrease.
 
     VertexImpl v5Impl = (VertexImpl) v5;
@@ -2016,10 +2015,10 @@ public class TestVertexImpl {
     EdgeManagerPlugin modifiedEdgeManager = v5Impl.sourceVertices.get(v3)
         .getEdgeManager();
     Assert.assertNotNull(modifiedEdgeManager);
-    assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
+    Assert.assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
 
     // Ensure initialize() is called with the correct payload
-    assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
+    Assert.assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
         ((EdgeManagerForTest) modifiedEdgeManager).getUserPayload().deepCopyAsArray()));
   }
 
@@ -2093,7 +2092,7 @@ public class TestVertexImpl {
     Assert.assertEquals(VertexTerminationCause.OWN_TASK_FAILURE, v.getTerminationCause());
     String diagnostics =
         StringUtils.join(v.getDiagnostics(), ",").toLowerCase();
-    assertTrue(diagnostics.contains("task failed"
+    Assert.assertTrue(diagnostics.contains("task failed"
         + ", taskid=" + t1.toString()));
   }
 
@@ -2105,7 +2104,7 @@ public class TestVertexImpl {
     String diagnostics =
         StringUtils.join(v2.getDiagnostics(), ",").toLowerCase();
     LOG.info("diagnostics v2: " + diagnostics);
-    assertTrue(diagnostics.contains(
+    Assert.assertTrue(diagnostics.contains(
         "vertex received kill in inited state"));
   }
 
@@ -2119,7 +2118,7 @@ public class TestVertexImpl {
     killVertex(v3);
     String diagnostics =
         StringUtils.join(v3.getDiagnostics(), ",").toLowerCase();
-    assertTrue(diagnostics.contains(
+    Assert.assertTrue(diagnostics.contains(
         "vertex received kill while in running state"));
   }
 
@@ -2201,7 +2200,7 @@ public class TestVertexImpl {
     Assert.assertNull(v2.getOutputCommitter("output"));
 
     VertexImpl v6 = vertices.get("vertex6");
-    assertTrue(v6.getOutputCommitter("outputx")
+    Assert.assertTrue(v6.getOutputCommitter("outputx")
         instanceof CountingOutputCommitter);
   }
 
@@ -2209,11 +2208,11 @@ public class TestVertexImpl {
   public void testVertexManagerInit() {
     initAllVertices(VertexState.INITED);
     VertexImpl v2 = vertices.get("vertex2");
-    assertTrue(v2.getVertexManager().getPlugin()
+    Assert.assertTrue(v2.getVertexManager().getPlugin()
         instanceof ImmediateStartVertexManager);
 
     VertexImpl v6 = vertices.get("vertex6");
-    assertTrue(v6.getVertexManager().getPlugin()
+    Assert.assertTrue(v6.getVertexManager().getPlugin()
         instanceof ShuffleVertexManager);
   }
 
@@ -3021,306 +3020,6 @@ public class TestVertexImpl {
     }
   }
 
-  @Test(timeout = 5000)
-  /**
-   * Ref: TEZ-1494
-   * If broadcast, one-to-one or custom edges are present in source, tasks should not start until
-   * 1 task from each source vertex is complete.
-   */
-  public void testTaskSchedulingWithCustomEdges() {
-    setupPreDagCreation();
-    dagPlan = createCustomDAGWithCustomEdges();
-    setupPostDagCreation();
-
-    /**
-     *
-     *   M2 --(SG)--> R3 --(B)--\
-     *                           \
-     *   M7 --(B)---------------->M5 ---(SG)--> R6
-     *                            /
-     *   M8---(C)--------------->/
-     */
-
-    //init M2
-    VertexImpl m2 = vertices.get("M2");
-    VertexImpl m7 = vertices.get("M7");
-    VertexImpl r3 = vertices.get("R3");
-    VertexImpl m5 = vertices.get("M5");
-    VertexImpl m8 = vertices.get("M8");
-
-    initVertex(m2);
-    initVertex(m7);
-    initVertex(m8);
-    assertTrue(m7.getState().equals(VertexState.INITED));
-    assertTrue(m5.getState().equals(VertexState.INITED));
-    assertTrue(m8.getState().equals(VertexState.INITED));
-    assertTrue(m7.getVertexManager().getPlugin() instanceof ImmediateStartVertexManager);
-
-    //Start M2; Let tasks complete in M2; Also let 1 task complete in R3
-    dispatcher.getEventHandler().handle(new VertexEvent(m2.getVertexId(), VertexEventType.V_START));
-    dispatcher.await();
-    VertexEventTaskAttemptCompleted taskAttemptCompleted = new VertexEventTaskAttemptCompleted
-        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m2.getVertexId(),0), 0), TaskAttemptStateInternal.SUCCEEDED);
-    VertexEventTaskCompleted taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m2
-        .getVertexId(), 0), TaskState.SUCCEEDED);
-    dispatcher.getEventHandler().handle(taskAttemptCompleted);
-    dispatcher.getEventHandler().handle(taskCompleted);
-    dispatcher.await();
-    taskAttemptCompleted = new VertexEventTaskAttemptCompleted
-        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(r3.getVertexId(),0), 0),
-            TaskAttemptStateInternal.SUCCEEDED);
-    taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(r3
-        .getVertexId(), 0), TaskState.SUCCEEDED);
-    dispatcher.getEventHandler().handle(taskAttemptCompleted);
-    dispatcher.getEventHandler().handle(taskCompleted);
-    dispatcher.await();
-    assertTrue(m2.getState().equals(VertexState.SUCCEEDED));
-    assertTrue(m5.numSuccessSourceAttemptCompletions == 1);
-    assertTrue(m5.getState().equals(VertexState.INITED));
-
-    //R3 should be in running state as it has one task completed, and rest are pending
-    assertTrue(r3.getState().equals(VertexState.RUNNING));
-
-    //Let us start M7; M5 should start not start as it is dependent on M8 as well
-    dispatcher.getEventHandler().handle(new VertexEvent(m7.getVertexId(),VertexEventType.V_START));
-    dispatcher.await();
-    //Let one of the tasks get over in M7 as well.
-    taskAttemptCompleted = new VertexEventTaskAttemptCompleted
-        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m7.getVertexId(),0), 0),
-            TaskAttemptStateInternal.SUCCEEDED);
-    taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m7
-        .getVertexId(), 0), TaskState.SUCCEEDED);
-    dispatcher.getEventHandler().handle(taskAttemptCompleted);
-    dispatcher.getEventHandler().handle(taskCompleted);
-    dispatcher.await();
-    assertTrue(m5.numSuccessSourceAttemptCompletions == 2);
-
-    //M5 should be in INITED state, as it depends on M8
-    assertTrue(m5.getState().equals(VertexState.INITED));
-    for(Task task : m5.getTasks().values()) {
-      assertTrue(task.getState().equals(TaskState.NEW));
-    }
-
-    //Let us start M8; M5 should start now
-    dispatcher.getEventHandler().handle(new VertexEvent(m8.getVertexId(),VertexEventType.V_START));
-    dispatcher.await();
-
-    //M5 in running state. But tasks should not be scheduled until M8 finishes a task.
-    assertTrue(m5.getState().equals(VertexState.RUNNING));
-    for(Task task : m5.getTasks().values()) {
-      assertTrue(task.getState().equals(TaskState.NEW));
-    }
-
-    //Let one of the tasks get over in M8 as well. This should trigger tasks to be scheduled in M5
-    taskAttemptCompleted = new VertexEventTaskAttemptCompleted
-        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m8.getVertexId(),0), 0),
-            TaskAttemptStateInternal.SUCCEEDED);
-    taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m8
-        .getVertexId(), 0), TaskState.SUCCEEDED);
-    dispatcher.getEventHandler().handle(taskAttemptCompleted);
-    dispatcher.getEventHandler().handle(taskCompleted);
-    dispatcher.await();
-
-    assertTrue(m5.numSuccessSourceAttemptCompletions == 3);
-    //Ensure all tasks in M5 are in scheduled state
-    for(Task task : m5.getTasks().values()) {
-      assertTrue(task.getState().equals(TaskState.SCHEDULED));
-    }
-  }
-
-  //For TEZ-1494
-  private DAGPlan createCustomDAGWithCustomEdges() {
-    /**
-     *
-     *   M2 --(SG)--> R3 --(B)--\
-     *                           \
-     *   M7 --(B)---------------->M5 ---(SG)--> R6
-     *                            /
-     *   M8---(C)--------------->/
-     */
-    DAGPlan dag = DAGPlan.newBuilder().setName("TestSamplerDAG")
-        .addVertex(VertexPlan.newBuilder()
-                .setName("M2")
-                .setProcessorDescriptor(
-                    TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
-                .setType(PlanVertexType.NORMAL)
-                .addTaskLocationHint(
-                    PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
-                .setTaskConfig(PlanTaskConfiguration.newBuilder()
-                        .setNumTasks(1)
-                        .setVirtualCores(4)
-                        .setMemoryMb(1024)
-                        .setJavaOpts("")
-                        .setTaskModule("M2.class")
-                        .build()
-                )
-                .addOutEdgeId("M2_R3")
-                .build()
-        )
-        .addVertex(VertexPlan.newBuilder()
-                .setName("M8")
-                .setProcessorDescriptor(
-                    TezEntityDescriptorProto.newBuilder().setClassName("M8.class"))
-                .setType(PlanVertexType.NORMAL)
-                .addTaskLocationHint(
-                    PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
-                .setTaskConfig(PlanTaskConfiguration.newBuilder()
-                        .setNumTasks(1)
-                        .setVirtualCores(4)
-                        .setMemoryMb(1024)
-                        .setJavaOpts("")
-                        .setTaskModule("M8.class")
-                        .build()
-                )
-                .addOutEdgeId("M8_M5")
-                .build()
-        )
-         .addVertex(VertexPlan.newBuilder()
-                 .setName("R3")
-                 .setProcessorDescriptor(
-                     TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
-                 .setType(PlanVertexType.NORMAL)
-                 .addTaskLocationHint(
-                     PlanTaskLocationHint.newBuilder().addHost("host2").addRack("rack1").build())
-                 .setTaskConfig(PlanTaskConfiguration.newBuilder()
-                         .setNumTasks(10)
-                         .setVirtualCores(4)
-                         .setMemoryMb(1024)
-                         .setJavaOpts("")
-                         .setTaskModule("R3.class")
-                         .build()
-                 )
-                 .addInEdgeId("M2_R3")
-                 .addOutEdgeId("R3_M5")
-                 .build()
-         )
-        .addVertex(VertexPlan.newBuilder()
-                .setName("M5")
-                .setProcessorDescriptor(
-                    TezEntityDescriptorProto.newBuilder().setClassName("M5.class"))
-                .setType(PlanVertexType.NORMAL)
-                .addTaskLocationHint(
-                    PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
-                .setTaskConfig(PlanTaskConfiguration.newBuilder()
-                        .setNumTasks(10)
-                        .setVirtualCores(4)
-                        .setMemoryMb(1024)
-                        .setJavaOpts("")
-                        .setTaskModule("M5.class")
-                        .build()
-                )
-                .addInEdgeId("R3_M5")
-                .addInEdgeId("M7_M5")
-                .addInEdgeId("M8_M5")
-                .addOutEdgeId("M5_R6")
-                .build()
-        )
-        .addVertex(VertexPlan.newBuilder()
-                .setName("M7")
-                .setProcessorDescriptor(
-                    TezEntityDescriptorProto.newBuilder().setClassName("M7.class"))
-                .setType(PlanVertexType.NORMAL)
-                .addTaskLocationHint(
-                    PlanTaskLocationHint.newBuilder().addHost("host4").addRack("rack1").build())
-                .setTaskConfig(PlanTaskConfiguration.newBuilder()
-                        .setNumTasks(10)
-                        .setVirtualCores(4)
-                        .setMemoryMb(1024)
-                        .setJavaOpts("")
-                        .setTaskModule("M7.class")
-                        .build()
-                )
-                .addOutEdgeId("M7_M5")
-                .build()
-        )
-        .addVertex(VertexPlan.newBuilder()
-                .setName("R6")
-                .setProcessorDescriptor(
-                    TezEntityDescriptorProto.newBuilder().setClassName("R6.class"))
-                .setType(PlanVertexType.NORMAL)
-                .addTaskLocationHint(
-                    PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
-                .setTaskConfig(PlanTaskConfiguration.newBuilder()
-                        .setNumTasks(1)
-                        .setVirtualCores(4)
-                        .setMemoryMb(1024)
-                        .setJavaOpts("")
-                        .setTaskModule("R6.class")
-                        .build()
-                )
-                .addInEdgeId("M5_R6")
-                .build()
-        )
-        .addEdge(
-            EdgePlan.newBuilder()
-                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3"))
-                .setInputVertexName("M2")
-                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3.class"))
-                .setOutputVertexName("R3")
-                .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
-                .setId("M2_R3")
-                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                .build()
-        )
-        .addEdge(
-            EdgePlan.newBuilder()
-                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5"))
-                .setInputVertexName("R3")
-                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5.class"))
-                .setOutputVertexName("M5")
-                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
-                .setId("R3_M5")
-                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                .build()
-        )
-        .addEdge(
-            EdgePlan.newBuilder()
-                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5"))
-                .setInputVertexName("M7")
-                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5.class"))
-                .setOutputVertexName("M5")
-                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
-                .setId("M7_M5")
-                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                .build()
-        )
-        .addEdge(
-            EdgePlan.newBuilder()
-                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6"))
-                .setInputVertexName("M5")
-                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6.class"))
-                .setOutputVertexName("R6")
-                .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
-                .setId("M5_R6")
-                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                .build()
-        )
-        .addEdge(
-            EdgePlan.newBuilder()
-                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5"))
-                .setInputVertexName("M8")
-                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5.class"))
-                .setEdgeManager(
-                    TezEntityDescriptorProto.newBuilder()
-                        .setClassName(EdgeManagerForTest.class.getName())
-                        .setUserPayload(ByteString.copyFrom(edgePayload))
-                        .build())
-                .setOutputVertexName("M5")
-                .setDataMovementType(PlanEdgeDataMovementType.CUSTOM)
-                .setId("M8_M5")
-                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
-                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
-                .build()
-        )
-        .build();
-
-    return dag;
-  }
-
   @SuppressWarnings("unchecked")
   @Test(timeout = 5000)
   public void testVertexWithInitializerSuccess() {
@@ -3516,7 +3215,6 @@ public class TestVertexImpl {
           dispatcher.getEventHandler(), taskAttemptListener,
           clock, thh, true, appContext, vertexLocationHint, null, taskSpecificLaunchCmdOption,
           updateTracker);
-      v.setInputVertices(new HashMap());
       vertexIdMap.put(vId, v);
       vertices.put(v.getName(), v);
       v.handle(new VertexEvent(vId, VertexEventType.V_INIT));
@@ -3756,8 +3454,8 @@ public class TestVertexImpl {
     List<GroupInputSpec> groupInSpec = vC.getGroupInputSpecList(0);
     Assert.assertEquals(1, groupInSpec.size());
     Assert.assertEquals("Group", groupInSpec.get(0).getGroupName());
-    assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
-    assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
+    Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
+    Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
     groupInSpec.get(0).getMergedInputDescriptor().getClassName().equals("Group.class");
   }
   


[19/25] git commit: TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless of DAG submission (bikas)

Posted by je...@apache.org.
TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless of DAG submission (bikas)


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

Branch: refs/heads/TEZ-8
Commit: 5d6b8fd5660dbe5828467d5678bbbd58c3702c7a
Parents: 7d1303f
Author: Bikas Saha <bi...@apache.org>
Authored: Mon Sep 15 11:51:02 2014 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Mon Sep 15 11:51:02 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  4 +
 .../java/org/apache/tez/client/TezClient.java   |  5 +-
 .../org/apache/tez/client/TezClientUtils.java   | 15 +++-
 .../main/java/org/apache/tez/dag/api/DAG.java   |  9 ++-
 .../org/apache/tez/client/TestTezClient.java    | 77 +++++++++++++-------
 .../org/apache/tez/dag/app/TestPreemption.java  |  4 +-
 6 files changed, 80 insertions(+), 34 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 3198323..1520e70 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,6 +19,8 @@ ALL CHANGES:
   TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
   TEZ-1524. Resolve user group information only if ACLs are enabled.
   TEZ-1581. GroupByOrderByMRRTest no longer functional.
+  TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
+  of DAG submission
 
 Release 0.5.1: Unreleased
 
@@ -36,6 +38,8 @@ ALL CHANGES
   TEZ-1519. TezTaskRunner should not initialize TezConfiguration in TezChild.
   TEZ-1534. Make client side configs available to AM and tasks.
   TEZ-1574. Support additional formats for the tez deployed archive
+  TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
+  of DAG submission
 
 Release 0.5.0: 2014-09-03
 

http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index 77ab20c..5cec8b0 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -366,10 +366,9 @@ public class TezClient {
     }
     
     Map<String, LocalResource> tezJarResources = getTezJarResources(sessionCredentials);
-    TezClientUtils.updateDAGVertices(dag, amConfig, tezJarResources,
+    DAGPlan dagPlan = TezClientUtils.prepareAndCreateDAGPlan(dag, amConfig, tezJarResources,
         TezClientUtils.usingTezLibsFromArchive(tezJarResources), sessionCredentials);
-    
-    DAGPlan dagPlan = dag.createDag(amConfig.getTezConfiguration());
+
     SubmitDAGRequestProto.Builder requestBuilder = SubmitDAGRequestProto.newBuilder();
     requestBuilder.setDAGPlan(dagPlan).build();
     if (!additionalLocalResources.isEmpty()) {

http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index 3297adc..917fcff 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -557,7 +557,7 @@ public class TezClientUtils {
 
     if(dag != null) {
       
-      updateDAGVertices(dag, amConfig, tezJarResources, tezLrsAsArchive, sessionCreds);
+      DAGPlan dagPB = prepareAndCreateDAGPlan(dag, amConfig, tezJarResources, tezLrsAsArchive, sessionCreds);
 
       // emit protobuf DAG file style
       Path binaryPath = TezCommonUtils.getTezBinPlanStagingPath(tezSysStagingPath);
@@ -567,8 +567,6 @@ public class TezClientUtils {
             + sessionJarsPath + " binaryPlanPath :" + binaryPath);
       }
 
-      DAGPlan dagPB = dag.createDag(amConfig.getTezConfiguration());
-
       FSDataOutputStream dagPBOutBinaryStream = null;
 
       try {
@@ -644,6 +642,17 @@ public class TezClientUtils {
     }
   }
   
+  static DAGPlan prepareAndCreateDAGPlan(DAG dag, AMConfiguration amConfig,
+      Map<String, LocalResource> tezJarResources, boolean tezLrsAsArchive,
+      Credentials credentials) throws IOException {
+    DAGPlan dagPB = dag.getCachedDAGPlan();
+    if (dagPB == null) {
+      updateDAGVertices(dag, amConfig, tezJarResources, tezLrsAsArchive, credentials);  
+      dagPB = dag.createDag(amConfig.getTezConfiguration());  
+    }    
+    return dagPB;
+  }
+  
   static void maybeAddDefaultLoggingJavaOpts(String logLevel, List<String> vargs) {
     if (vargs != null && !vargs.isEmpty()) {
       for (String arg : vargs) {

http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
index d063e79..ffd2e83 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DAG.java
@@ -89,6 +89,7 @@ public class DAG {
   Map<String, LocalResource> commonTaskLocalFiles = Maps.newHashMap();
   
   private Stack<String> topologicalVertexStack = new Stack<String>();
+  private DAGPlan cachedDAGPlan;
 
   private DAG(String name) {
     this.name = name;
@@ -574,6 +575,10 @@ public class DAG {
     }
   }
 
+  @Private
+  public DAGPlan getCachedDAGPlan() {
+    return cachedDAGPlan;
+  }
 
   // create protobuf message describing DAG
   @Private
@@ -787,6 +792,8 @@ public class DAG {
       dagBuilder.setCredentialsBinary(DagTypeConverters.convertCredentialsToProto(credentials));
       TezCommonUtils.logCredentials(LOG, credentials, "dag");
     }
-    return dagBuilder.build();
+    
+    cachedDAGPlan = dagBuilder.build();
+    return cachedDAGPlan;
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
index 5044d55..fbb4486 100644
--- a/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
+++ b/tez-api/src/test/java/org/apache/tez/client/TestTezClient.java
@@ -20,6 +20,7 @@ package org.apache.tez.client;
 
 import java.io.IOException;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.Map;
 import java.util.concurrent.atomic.AtomicReference;
 
@@ -89,7 +90,16 @@ public class TestTezClient {
     }
   }
   
-  static void configure(TezClientForTest client) throws YarnException, IOException {
+  TezClientForTest configure() throws YarnException, IOException {
+    return configure(new HashMap<String, LocalResource>(), true);
+  }
+  
+  TezClientForTest configure(Map<String, LocalResource> lrs, boolean isSession) throws YarnException, IOException {
+    TezConfiguration conf = new TezConfiguration();
+    conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
+    conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, isSession);
+    TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
+
     ApplicationId appId1 = ApplicationId.newInstance(0, 1);
     YarnClient yarnClient = mock(YarnClient.class, RETURNS_DEEP_STUBS);
     when(yarnClient.createApplication().getNewApplicationResponse().getApplicationId()).thenReturn(appId1);
@@ -100,31 +110,27 @@ public class TestTezClient {
     client.mockTezYarnClient = new TezYarnClient(yarnClient);
     client.mockYarnClient = yarnClient;
     client.mockAppId = appId1;
+    
+    return client;    
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testTezclientApp() throws Exception {
     testTezClient(false);
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testTezclientSession() throws Exception {
     testTezClient(true);
   }
   
   public void testTezClient(boolean isSession) throws Exception {
-    TezConfiguration conf = new TezConfiguration();
-    conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
-    conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, isSession);
-
     Map<String, LocalResource> lrs = Maps.newHashMap();
     String lrName1 = "LR1";
     lrs.put(lrName1, LocalResource.newInstance(URL.newInstance("file:///", "localhost", 0, "test"),
         LocalResourceType.FILE, LocalResourceVisibility.PUBLIC, 1, 1));
     
-    TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
-    
-    configure(client);
+    TezClientForTest client = configure(lrs, isSession);
     
     ArgumentCaptor<ApplicationSubmissionContext> captor = ArgumentCaptor.forClass(ApplicationSubmissionContext.class);
     client.start();
@@ -229,13 +235,7 @@ public class TestTezClient {
   }
   
   public void testPreWarm() throws Exception {
-    TezConfiguration conf = new TezConfiguration();
-    conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
-    conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
-    Map<String, LocalResource> lrs = Maps.newHashMap();
-    final TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
-
-    configure(client);
+    TezClientForTest client = configure();
     client.start();
     PreWarmVertex vertex = PreWarmVertex.create("PreWarm", 1, Resource.newInstance(1, 1));
     client.preWarm(vertex);
@@ -248,16 +248,42 @@ public class TestTezClient {
     client.stop();
   }
   
+  @Test (timeout = 10000)
+  public void testMultipleSubmissions() throws Exception {
+    testMultipleSubmissionsJob(false);
+    testMultipleSubmissionsJob(true);
+  }
+  
+  public void testMultipleSubmissionsJob(boolean isSession) throws Exception {
+    TezClientForTest client1 = configure(new HashMap<String, LocalResource>(), isSession);
+    client1.start();
+    
+    String mockLR1Name = "LR1";
+    Map<String, LocalResource> lrDAG = Collections.singletonMap(mockLR1Name, LocalResource
+        .newInstance(URL.newInstance("file:///", "localhost", 0, "test"), LocalResourceType.FILE,
+            LocalResourceVisibility.PUBLIC, 1, 1));
+    Vertex vertex = Vertex.create("Vertex", ProcessorDescriptor.create("P"), 1,
+        Resource.newInstance(1, 1));
+    DAG dag = DAG.create("DAG").addVertex(vertex).addTaskLocalFiles(lrDAG);
+    // the dag resource will be added to the vertex once
+    client1.submitDAG(dag);
+    
+    TezClientForTest client2 = configure();
+    client2.start();
+    
+    // verify resubmission of same dag to new client (simulates submission error resulting in the
+    // creation of a new client and resubmission of the DAG)
+    // TEZ-1563 dag resource will not be added again to the vertex because its cached
+    // so resubmission works fine
+    client2.submitDAG(dag);
+    
+    client1.stop();
+    client2.stop();
+  }
+  
   @Test(timeout = 5000)
   public void testWaitTillReady_Interrupt() throws Exception {
-    TezConfiguration conf = new TezConfiguration();
-    conf.setBoolean(TezConfiguration.TEZ_IGNORE_LIB_URIS, true);
-    conf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
-    Map<String, LocalResource> lrs = Maps.newHashMap();
-    final TezClientForTest client = new TezClientForTest("test", conf, lrs, null);
-
-    configure(client);
-
+    final TezClientForTest client = configure();
     client.start();
 
     when(client.mockYarnClient.getApplicationReport(client.mockAppId).getYarnApplicationState())
@@ -278,6 +304,7 @@ public class TestTezClient {
     thread.interrupt();
     thread.join();
     Assert.assertThat(exceptionReference.get(),CoreMatchers. instanceOf(InterruptedException.class));
+    client.stop();
   }
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/5d6b8fd5/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
index c7aacd4..cfb1c9f 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
@@ -85,7 +85,7 @@ public class TestPreemption {
     return dag;
   }
   
-  @Test
+  @Test (timeout = 5000)
   public void testPreemptionWithoutSession() throws Exception {
     System.out.println("TestPreemptionWithoutSession");
     TezConfiguration tezconf = new TezConfiguration(defaultConf);
@@ -120,7 +120,7 @@ public class TestPreemption {
     tezClient.stop();
   }
   
-  @Test
+  @Test (timeout = 30000)
   public void testPreemptionWithSession() throws Exception {
     System.out.println("TestPreemptionWithSession");
     MockTezClient tezClient = createTezSession();


[18/25] git commit: TEZ-1581. GroupByOrderByMRRTest no longer functional. (hitesh)

Posted by je...@apache.org.
TEZ-1581. GroupByOrderByMRRTest no longer functional. (hitesh)


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

Branch: refs/heads/TEZ-8
Commit: 7d1303fa606e700bd4d4b2a122a73a5badbbc889
Parents: edb841c
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 15:18:05 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 15:18:05 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../examples/GroupByOrderByMRRTest.java         | 283 +++++++++++++------
 2 files changed, 205 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/7d1303fa/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 59be260..3198323 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -18,6 +18,7 @@ ALL CHANGES:
   TEZ-1569. Add tests for preemption
   TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
   TEZ-1524. Resolve user group information only if ACLs are enabled.
+  TEZ-1581. GroupByOrderByMRRTest no longer functional.
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/7d1303fa/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
index 939bea0..393faea 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/GroupByOrderByMRRTest.java
@@ -18,34 +18,54 @@
 
 package org.apache.tez.mapreduce.examples;
 
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
 import java.util.StringTokenizer;
+import java.util.TreeMap;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.fs.FileAlreadyExistsException;
+import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IntWritable;
 import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobID;
+import org.apache.hadoop.mapred.JobConf;
 import org.apache.hadoop.mapreduce.Mapper;
 import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.TypeConverter;
 import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
+import org.apache.hadoop.mapreduce.lib.input.TextInputFormat;
+import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.util.GenericOptionsParser;
 import org.apache.hadoop.util.Tool;
 import org.apache.hadoop.util.ToolRunner;
-import org.apache.hadoop.yarn.api.records.ApplicationId;
-import org.apache.tez.client.MRTezClient;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.common.TezUtils;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.DataSourceDescriptor;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.ProcessorDescriptor;
 import org.apache.tez.dag.api.TezConfiguration;
-import org.apache.tez.dag.api.TezException;
+import org.apache.tez.dag.api.UserPayload;
+import org.apache.tez.dag.api.Vertex;
 import org.apache.tez.dag.api.client.DAGClient;
 import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.mapreduce.hadoop.MRHelpers;
+import org.apache.tez.mapreduce.hadoop.MRInputHelpers;
 import org.apache.tez.mapreduce.hadoop.MRJobConfig;
-import org.apache.tez.mapreduce.hadoop.MultiStageMRConfigUtil;
+import org.apache.tez.mapreduce.output.MROutputLegacy;
+import org.apache.tez.mapreduce.processor.map.MapProcessor;
+import org.apache.tez.mapreduce.processor.reduce.ReduceProcessor;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
+import org.apache.tez.runtime.library.conf.OrderedPartitionedKVEdgeConfig;
+import org.apache.tez.runtime.library.partitioner.HashPartitioner;
 
 /**
  * Simple example that does a GROUP BY ORDER BY in an MRR job
@@ -94,7 +114,7 @@ public class GroupByOrderByMRRTest extends Configured implements Tool {
     public void map(Object key, Text value, Context context
         ) throws IOException, InterruptedException {
       StringTokenizer itr = new StringTokenizer(value.toString());
-      String empName = "";
+      String empName;
       String deptName = "";
       if (itr.hasMoreTokens()) {
         empName = itr.nextToken();
@@ -149,25 +169,141 @@ public class GroupByOrderByMRRTest extends Configured implements Tool {
     }
   }
 
+  private static DAG createDAG(Configuration conf, Map<String, LocalResource> commonLocalResources,
+      Path stagingDir, String inputPath, String outputPath, boolean useMRSettings)
+      throws Exception {
+
+    Configuration mapStageConf = new JobConf(conf);
+    mapStageConf.set(MRJobConfig.MAP_CLASS_ATTR,
+        MyMapper.class.getName());
+
+    MRHelpers.translateMRConfToTez(mapStageConf);
+
+    Configuration iReduceStageConf = new JobConf(conf);
+    // TODO replace with auto-reduce parallelism
+    iReduceStageConf.setInt(MRJobConfig.NUM_REDUCES, 2);
+    iReduceStageConf.set(MRJobConfig.REDUCE_CLASS_ATTR,
+        MyGroupByReducer.class.getName());
+    iReduceStageConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, Text.class.getName());
+    iReduceStageConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS,
+        IntWritable.class.getName());
+    iReduceStageConf.setBoolean("mapred.mapper.new-api", true);
+    MRHelpers.translateMRConfToTez(iReduceStageConf);
+
+    Configuration finalReduceConf = new JobConf(conf);
+    finalReduceConf.setInt(MRJobConfig.NUM_REDUCES, 1);
+    finalReduceConf.set(MRJobConfig.REDUCE_CLASS_ATTR,
+        MyOrderByNoOpReducer.class.getName());
+    finalReduceConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_KEY_CLASS, IntWritable.class.getName());
+    finalReduceConf.set(TezRuntimeConfiguration.TEZ_RUNTIME_VALUE_CLASS, Text.class.getName());
+    MRHelpers.translateMRConfToTez(finalReduceConf);
+
+    MRHelpers.configureMRApiUsage(mapStageConf);
+    MRHelpers.configureMRApiUsage(iReduceStageConf);
+    MRHelpers.configureMRApiUsage(finalReduceConf);
+
+    List<Vertex> vertices = new ArrayList<Vertex>();
+
+    ByteArrayOutputStream outputStream = new ByteArrayOutputStream(4096);
+    mapStageConf.writeXml(outputStream);
+    String mapStageHistoryText = new String(outputStream.toByteArray(), "UTF-8");
+    mapStageConf.set(MRJobConfig.INPUT_FORMAT_CLASS_ATTR,
+        TextInputFormat.class.getName());
+    mapStageConf.set(FileInputFormat.INPUT_DIR, inputPath);
+    mapStageConf.setBoolean("mapred.mapper.new-api", true);
+    DataSourceDescriptor dsd = MRInputHelpers.configureMRInputWithLegacySplitGeneration(
+        mapStageConf, stagingDir, true);
+
+    Vertex mapVertex;
+    ProcessorDescriptor mapProcessorDescriptor =
+        ProcessorDescriptor.create(MapProcessor.class.getName())
+            .setUserPayload(
+                TezUtils.createUserPayloadFromConf(mapStageConf))
+            .setHistoryText(mapStageHistoryText);
+    if (!useMRSettings) {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor);
+    } else {
+      mapVertex = Vertex.create("initialmap", mapProcessorDescriptor, -1,
+          MRHelpers.getResourceForMRMapper(mapStageConf));
+      mapVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
+    }
+    mapVertex.addTaskLocalFiles(commonLocalResources)
+        .addDataSource("MRInput", dsd);
+    vertices.add(mapVertex);
+
+    ByteArrayOutputStream iROutputStream = new ByteArrayOutputStream(4096);
+    iReduceStageConf.writeXml(iROutputStream);
+    String iReduceStageHistoryText = new String(iROutputStream.toByteArray(), "UTF-8");
+
+    ProcessorDescriptor iReduceProcessorDescriptor = ProcessorDescriptor.create(
+        ReduceProcessor.class.getName())
+        .setUserPayload(TezUtils.createUserPayloadFromConf(iReduceStageConf))
+        .setHistoryText(iReduceStageHistoryText);
+
+    Vertex intermediateVertex;
+    if (!useMRSettings) {
+      intermediateVertex = Vertex.create("ireduce1", iReduceProcessorDescriptor, 1);
+    } else {
+      intermediateVertex = Vertex.create("ireduce1", iReduceProcessorDescriptor,
+          1, MRHelpers.getResourceForMRReducer(iReduceStageConf));
+      intermediateVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(iReduceStageConf));
+    }
+    intermediateVertex.addTaskLocalFiles(commonLocalResources);
+    vertices.add(intermediateVertex);
+
+    ByteArrayOutputStream finalReduceOutputStream = new ByteArrayOutputStream(4096);
+    finalReduceConf.writeXml(finalReduceOutputStream);
+    String finalReduceStageHistoryText = new String(finalReduceOutputStream.toByteArray(), "UTF-8");
+    UserPayload finalReducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
+    Vertex finalReduceVertex;
+
+    ProcessorDescriptor finalReduceProcessorDescriptor =
+        ProcessorDescriptor.create(
+            ReduceProcessor.class.getName())
+            .setUserPayload(finalReducePayload)
+            .setHistoryText(finalReduceStageHistoryText);
+    if (!useMRSettings) {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1);
+    } else {
+      finalReduceVertex = Vertex.create("finalreduce", finalReduceProcessorDescriptor, 1,
+          MRHelpers.getResourceForMRReducer(finalReduceConf));
+      finalReduceVertex.setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
+    }
+    finalReduceVertex.addTaskLocalFiles(commonLocalResources);
+    finalReduceVertex.addDataSink("MROutput",
+        MROutputLegacy.createConfigBuilder(finalReduceConf, TextOutputFormat.class, outputPath)
+            .build());
+    vertices.add(finalReduceVertex);
+
+    DAG dag = DAG.create("groupbyorderbymrrtest");
+    for (Vertex v : vertices) {
+      dag.addVertex(v);
+    }
+
+    OrderedPartitionedKVEdgeConfig edgeConf1 = OrderedPartitionedKVEdgeConfig
+        .newBuilder(Text.class.getName(), IntWritable.class.getName(),
+            HashPartitioner.class.getName()).setFromConfiguration(conf)
+        .configureInput().useLegacyInput().done().build();
+    dag.addEdge(
+        Edge.create(dag.getVertex("initialmap"), dag.getVertex("ireduce1"),
+            edgeConf1.createDefaultEdgeProperty()));
+
+    OrderedPartitionedKVEdgeConfig edgeConf2 = OrderedPartitionedKVEdgeConfig
+        .newBuilder(IntWritable.class.getName(), Text.class.getName(),
+            HashPartitioner.class.getName()).setFromConfiguration(conf)
+        .configureInput().useLegacyInput().done().build();
+    dag.addEdge(
+        Edge.create(dag.getVertex("ireduce1"), dag.getVertex("finalreduce"),
+            edgeConf2.createDefaultEdgeProperty()));
+
+    return dag;
+  }
+
+
   @Override
   public int run(String[] args) throws Exception {
     Configuration conf = getConf();
 
-    // Configure intermediate reduces
-    conf.setInt(MRJobConfig.MRR_INTERMEDIATE_STAGES, 1);
-
-    // Set reducer class for intermediate reduce
-    conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.job.reduce.class"), MyGroupByReducer.class, Reducer.class);
-    // Set reducer output key class
-    conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.map.output.key.class"), IntWritable.class, Object.class);
-    // Set reducer output value class
-    conf.setClass(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.map.output.value.class"), Text.class, Object.class);
-    conf.setInt(MultiStageMRConfigUtil.getPropertyNameForIntermediateStage(1,
-        "mapreduce.job.reduces"), 2);
-
     String[] otherArgs = new GenericOptionsParser(conf, args).
         getRemainingArgs();
     if (otherArgs.length != 2) {
@@ -176,66 +312,55 @@ public class GroupByOrderByMRRTest extends Configured implements Tool {
       return 2;
     }
 
-    @SuppressWarnings("deprecation")
-    Job job = new Job(conf, "groupbyorderbymrrtest");
-
-    job.setJarByClass(GroupByOrderByMRRTest.class);
-
-    // Configure map
-    job.setMapperClass(MyMapper.class);
-    job.setMapOutputKeyClass(Text.class);
-    job.setMapOutputValueClass(IntWritable.class);
-
-    // Configure reduce
-    job.setReducerClass(MyOrderByNoOpReducer.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(IntWritable.class);
-    job.setNumReduceTasks(1);
-
-    FileInputFormat.addInputPath(job, new Path(otherArgs[0]));
-    FileOutputFormat.setOutputPath(job, new Path(otherArgs[1]));
-
-    job.submit();
-    JobID jobId = job.getJobID();
-    ApplicationId appId = TypeConverter.toYarn(jobId).getAppId();
-
-    DAGClient dagClient = MRTezClient.getDAGClient(appId, new TezConfiguration(conf), null);
-    DAGStatus dagStatus;
-    String[] vNames = { "initialmap" , "ireduce1" , "finalreduce" };
-    while (true) {
-      dagStatus = dagClient.getDAGStatus(null);
-      if(dagStatus.getState() == DAGStatus.State.RUNNING ||
-         dagStatus.getState() == DAGStatus.State.SUCCEEDED ||
-         dagStatus.getState() == DAGStatus.State.FAILED ||
-         dagStatus.getState() == DAGStatus.State.KILLED ||
-         dagStatus.getState() == DAGStatus.State.ERROR) {
-        break;
-      }
-      try {
-        Thread.sleep(500);
-      } catch (InterruptedException e) {
-        // continue;
-      }
+    String inputPath = otherArgs[0];
+    String outputPath = otherArgs[1];
+
+    UserGroupInformation.setConfiguration(conf);
+
+    TezConfiguration tezConf = new TezConfiguration(conf);
+    FileSystem fs = FileSystem.get(conf);
+
+    if (fs.exists(new Path(outputPath))) {
+      throw new FileAlreadyExistsException("Output directory "
+          + outputPath + " already exists");
     }
 
-    while (dagStatus.getState() == DAGStatus.State.RUNNING) {
-      try {
-        ExampleDriver.printDAGStatus(dagClient, vNames);
-        try {
-          Thread.sleep(1000);
-        } catch (InterruptedException e) {
-          // continue;
-        }
-        dagStatus = dagClient.getDAGStatus(null);
-      } catch (TezException e) {
-        LOG.fatal("Failed to get application progress. Exiting");
+    Map<String, LocalResource> localResources =
+        new TreeMap<String, LocalResource>();
+
+    String stagingDirStr =  conf.get(TezConfiguration.TEZ_AM_STAGING_DIR,
+        TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT) + Path.SEPARATOR +
+        Long.toString(System.currentTimeMillis());
+    Path stagingDir = new Path(stagingDirStr);
+    FileSystem pathFs = stagingDir.getFileSystem(tezConf);
+    pathFs.mkdirs(new Path(stagingDirStr));
+
+    tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirStr);
+    stagingDir = pathFs.makeQualified(new Path(stagingDirStr));
+
+    TezClient tezClient = TezClient.create("groupbyorderbymrrtest", tezConf);
+    tezClient.start();
+
+    LOG.info("Submitting groupbyorderbymrrtest DAG as a new Tez Application");
+
+    try {
+      DAG dag = createDAG(conf, localResources, stagingDir, inputPath, outputPath, true);
+
+      tezClient.waitTillReady();
+
+      DAGClient dagClient = tezClient.submitDAG(dag);
+
+      DAGStatus dagStatus = dagClient.waitForCompletionWithStatusUpdates(null);
+      if (dagStatus.getState() != DAGStatus.State.SUCCEEDED) {
+        LOG.error("groupbyorderbymrrtest failed, state=" + dagStatus.getState()
+            + ", diagnostics=" + dagStatus.getDiagnostics());
         return -1;
       }
+      LOG.info("Application completed. " + "FinalState=" + dagStatus.getState());
+      return 0;
+    } finally {
+      tezClient.stop();
     }
-
-    ExampleDriver.printDAGStatus(dagClient, vNames);
-    LOG.info("Application completed. " + "FinalState=" + dagStatus.getState());
-    return dagStatus.getState() == DAGStatus.State.SUCCEEDED ? 0 : 1;
   }
 
   public static void main(String[] args) throws Exception {


[21/25] git commit: TEZ-1585. Memory leak in tez session mode (Rajesh Balamohan)

Posted by je...@apache.org.
TEZ-1585. Memory leak in tez session mode (Rajesh Balamohan)


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

Branch: refs/heads/TEZ-8
Commit: 938097d85f6f943a399c16fa50f07a7e2b05bdb5
Parents: dbe6a38
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Tue Sep 16 11:51:13 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Tue Sep 16 11:51:13 2014 +0530

----------------------------------------------------------------------
 .../src/main/java/org/apache/tez/dag/app/DAGAppMaster.java    | 7 +++++++
 1 file changed, 7 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/938097d8/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 ea0ab3b..0e4f78b 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
@@ -536,6 +536,13 @@ public class DAGAppMaster extends AbstractService {
           }
         }
       }
+      //close all fs related caches
+      try {
+        FileSystem.closeAllForUGI(context.getCurrentDAG().getDagUGI());
+      } catch (IOException e) {
+        LOG.warn("Error occurred when trying to close FileSystem for userName " + context
+            .getCurrentDAG().getDagUGI().getUserName(), e);
+      }
       break;
     case AM_REBOOT:
       LOG.info("Received an AM_REBOOT signal");


[09/25] git commit: TEZ-1539. Change InputInitializerEvent semantics to SEND_ONCE_ON_TASK_SUCCESS. (sseth)

Posted by je...@apache.org.
TEZ-1539. Change InputInitializerEvent semantics to
SEND_ONCE_ON_TASK_SUCCESS. (sseth)


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

Branch: refs/heads/TEZ-8
Commit: b4580a7b81ab1a5619987296641ebbf50fda4c55
Parents: fb05aac
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 12 02:37:11 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 12 02:37:11 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   4 +
 .../tez/runtime/api/InputInitializer.java       |   5 +-
 .../api/events/InputInitializerEvent.java       |  23 +
 .../org/apache/tez/dag/app/RecoveryParser.java  |   8 +-
 .../app/dag/RootInputInitializerManager.java    | 194 +++++-
 .../tez/dag/app/dag/StateChangeNotifier.java    |  76 ++-
 .../dag/app/dag/TaskStateUpdateListener.java    |  35 +
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |  45 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |  75 ++-
 .../VertexDataMovementEventsGeneratedEvent.java | 214 ------
 .../VertexRecoverableEventsGeneratedEvent.java  | 223 +++++++
 tez-dag/src/main/proto/HistoryEvents.proto      |   1 +
 .../tez/dag/app/dag/impl/TestTaskImpl.java      |  10 +-
 .../tez/dag/app/dag/impl/TestTaskRecovery.java  |   3 +-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    | 648 ++++++++++++++++++-
 .../dag/app/dag/impl/TestVertexRecovery.java    |  16 +-
 .../TestHistoryEventsProtoConversion.java       |  10 +-
 .../impl/TestHistoryEventJsonConversion.java    |   4 +-
 .../ats/TestHistoryEventTimelineConversion.java |   4 +-
 .../org/apache/tez/test/TestDAGRecovery.java    |   6 +-
 20 files changed, 1290 insertions(+), 314 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f29d48d..dd20f43 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,10 @@ ALL CHANGES:
 
 Release 0.5.1: Unreleased
 
+INCOMPATIBLE CHANGES
+  TEZ-1539. Change InputInitializerEvent semantics to SEND_ONCE_ON_TASK_SUCCESS
+  TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
+
 ALL CHANGES
   TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput()
   TEZ-1515. Remove usage of ResourceBundles in Counters.

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
index 00896de..7b22b62 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/InputInitializer.java
@@ -93,7 +93,10 @@ public abstract class InputInitializer {
    * State changes will be received based on the registration via {@link
    * org.apache.tez.runtime.api.InputInitializerContext#registerForVertexStateUpdates(String,
    * java.util.Set)}. Notifications will be received for all registered state changes, and not just
-   * for the latest state update. They will be in order in which the state change occurred.
+   * for the latest state update. They will be in order in which the state change occurred. </p>
+   *
+   * Extensive processing should not be performed via this method call. Instead this should just be
+   * used as a notification mechanism to the main initialization, which is via the initialize method.
    *
    * @param stateUpdate an event indicating the name of the vertex, and it's updated state.
    *                    Additional information may be available for specific events, Look at the

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
index 3c5e78e..8360447 100644
--- a/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
+++ b/tez-api/src/main/java/org/apache/tez/runtime/api/events/InputInitializerEvent.java
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 
 import com.google.common.base.Preconditions;
 
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceAudience.Public;
 import org.apache.hadoop.classification.InterfaceStability.Unstable;
 import org.apache.tez.runtime.api.InputInitializer;
@@ -32,6 +33,13 @@ import org.apache.tez.runtime.api.Event;
 /**
  * An event that is routed to the specified {@link InputInitializer}.
  * This can be used to send information/metadata to the {@link InputInitializer}
+ * <p/>
+ *
+ * These events are routed to the InputInitializer, only after the task which generated the event
+ * succeeds. Also, the events will only be sent once per task - irrespective of how many attempts
+ * were run, or succeeded. An example of this is when a task is retried because the node on which it
+ * was running failed. If the Task had succeeded once, the event would already have been sent - and
+ * will not be resent when the task reruns and succeeds. </p>
  */
 @Unstable
 @Public
@@ -41,6 +49,7 @@ public class InputInitializerEvent extends Event {
   private String targetInputName;
 
   private ByteBuffer eventPayload;
+  private String sourceVertexName;
 
   private InputInitializerEvent(String targetVertexName, String targetInputName,
                                 ByteBuffer eventPayload) {
@@ -88,4 +97,18 @@ public class InputInitializerEvent extends Event {
   public ByteBuffer getUserPayload() {
     return eventPayload == null ? null : eventPayload.asReadOnlyBuffer();
   }
+
+  @InterfaceAudience.Private
+  public void setSourceVertexName(String srcVertexName) {
+    this.sourceVertexName = srcVertexName;
+  }
+
+  /**
+   * Returns the name of the vertex which generated the event. This will only be populated after
+   * the event has been routed by the AM.
+   * @return the name of the source vertex
+   */
+  public String getSourceVertexName() {
+    return this.sourceVertexName;
+  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 9ba5847..85851c5 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
@@ -58,7 +58,7 @@ 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.VertexDataMovementEventsGeneratedEvent;
+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;
@@ -207,7 +207,7 @@ public class RecoveryParser {
         event = new TaskAttemptFinishedEvent();
         break;
       case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-        event = new VertexDataMovementEventsGeneratedEvent();
+        event = new VertexRecoverableEventsGeneratedEvent();
         break;
       default:
         throw new IOException("Invalid data found, unknown event type "
@@ -865,8 +865,8 @@ public class RecoveryParser {
               + ", eventType=" + eventType
               + ", event=" + event.toString());
           assert recoveredDAGData.recoveredDAG != null;
-          VertexDataMovementEventsGeneratedEvent vEvent =
-              (VertexDataMovementEventsGeneratedEvent) event;
+          VertexRecoverableEventsGeneratedEvent vEvent =
+              (VertexRecoverableEventsGeneratedEvent) event;
           Vertex v = recoveredDAGData.recoveredDAG.getVertex(vEvent.getVertexID());
           v.restoreFromEvent(vEvent);
           break;

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
index 770761e..87d4eb6 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/RootInputInitializerManager.java
@@ -20,7 +20,10 @@ package org.apache.tez.dag.app.dag;
 
 import javax.annotation.Nullable;
 import java.security.PrivilegedExceptionAction;
+import java.util.Collection;
 import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -30,10 +33,13 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicBoolean;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.LinkedListMultimap;
+import com.google.common.collect.ListMultimap;
 import com.google.common.collect.Lists;
 
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.tez.common.ReflectionUtils;
@@ -43,10 +49,12 @@ import org.apache.tez.dag.api.RootInputLeafOutput;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.event.*;
 import org.apache.tez.dag.api.event.VertexState;
+import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.app.AppContext;
 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.impl.TezRootInputInitializerContextImpl;
+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;
@@ -61,6 +69,7 @@ import com.google.common.util.concurrent.MoreExecutors;
 import com.google.common.util.concurrent.ThreadFactoryBuilder;
 
 import org.apache.tez.runtime.api.events.InputInitializerEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
 
 public class RootInputInitializerManager {
 
@@ -77,7 +86,8 @@ public class RootInputInitializerManager {
   private final Vertex vertex;
   private final AppContext appContext;
 
-  private final Map<String, InitializerWrapper> initializerMap = new HashMap<String, InitializerWrapper>();
+  @VisibleForTesting
+  final Map<String, InitializerWrapper> initializerMap = new HashMap<String, InitializerWrapper>();
 
   public RootInputInitializerManager(Vertex vertex, AppContext appContext,
                                      UserGroupInformation dagUgi, StateChangeNotifier stateTracker) {
@@ -100,7 +110,7 @@ public class RootInputInitializerManager {
       InputInitializer initializer = createInitializer(input, context);
 
       InitializerWrapper initializerWrapper =
-          new InitializerWrapper(input, initializer, context, vertex, entityStateTracker);
+          new InitializerWrapper(input, initializer, context, vertex, entityStateTracker, appContext);
       initializerMap.put(input.getName(), initializerWrapper);
       ListenableFuture<List<Event>> future = executor
           .submit(new InputInitializerCallable(initializerWrapper, dagUgi));
@@ -117,31 +127,40 @@ public class RootInputInitializerManager {
     return initializer;
   }
 
-  public void handleInitializerEvent(InputInitializerEvent event) {
-    Preconditions.checkState(vertex.getName().equals(event.getTargetVertexName()),
-        "Received event for incorrect vertex");
-    Preconditions.checkNotNull(event.getTargetInputName(), "target input name must be set");
-    InitializerWrapper initializer = initializerMap.get(event.getTargetInputName());
-    Preconditions.checkState(initializer != null,
-        "Received event for unknown input : " + event.getTargetInputName());
+  public void handleInitializerEvents(List<TezEvent> events) {
+    ListMultimap<InitializerWrapper, TezEvent> eventMap = LinkedListMultimap.create();
+
+    for (TezEvent tezEvent : events) {
+      Preconditions.checkState(tezEvent.getEvent() instanceof InputInitializerEvent);
+      InputInitializerEvent event = (InputInitializerEvent)tezEvent.getEvent();
+      Preconditions.checkState(vertex.getName().equals(event.getTargetVertexName()),
+          "Received event for incorrect vertex");
+      Preconditions.checkNotNull(event.getTargetInputName(), "target input name must be set");
+      InitializerWrapper initializer = initializerMap.get(event.getTargetInputName());
+      Preconditions.checkState(initializer != null,
+          "Received event for unknown input : " + event.getTargetInputName());
+      eventMap.put(initializer, tezEvent);
+    }
+
     // This is a restriction based on current flow - i.e. events generated only by initialize().
     // TODO Rework the flow as per the first comment on TEZ-1076
     if (isStopped) {
       LOG.warn("InitializerManager already stopped for " + vertex.getLogIdentifier() +
-          " Dropping event. [" + event + "]");
-      return;
+          " Dropping " + events.size() + " events");
     }
-    if (initializer.isComplete()) {
-      LOG.warn(
-          "Event targeted at vertex " + vertex.getLogIdentifier() + ", initializerWrapper for Input: " +
-              initializer.getInput().getName() +
-              " will be dropped, since Input has already been initialized. [" + event + "]");
-    }
-    try {
-      initializer.getInitializer().handleInputInitializerEvent(Lists.newArrayList(event));
-    } catch (Exception e) {
-      throw new TezUncheckedException(
-          "Initializer for input: " + event.getTargetInputName() + " failed to process event", e);
+
+    for (Map.Entry<InitializerWrapper, Collection<TezEvent>> entry : eventMap.asMap().entrySet()) {
+      InitializerWrapper initializerWrapper = entry.getKey();
+      if (initializerWrapper.isComplete()) {
+        LOG.warn(entry.getValue().size() +
+            " events targeted at vertex " + vertex.getLogIdentifier() +
+            ", initializerWrapper for Input: " +
+            initializerWrapper.getInput().getName() +
+            " will be dropped, since Input has already been initialized.");
+      } else {
+        initializerWrapper.handleInputInitializerEvents(entry.getValue());
+      }
+
     }
   }
 
@@ -157,7 +176,13 @@ public class RootInputInitializerManager {
   protected InputInitializerCallback createInputInitializerCallback(InitializerWrapper initializer) {
     return new InputInitializerCallback(initializer, eventHandler, vertex.getVertexId());
   }
-  
+
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  public InitializerWrapper getInitializerWrapper(String inputName) {
+    return initializerMap.get(inputName);
+  }
+
   public void shutdown() {
     if (executor != null && !isStopped) {
       // Don't really care about what is running if an error occurs. If no error
@@ -232,7 +257,9 @@ public class RootInputInitializerManager {
     }
   }
 
-  private static class InitializerWrapper implements VertexStateUpdateListener {
+  @VisibleForTesting
+  @InterfaceAudience.Private
+  public static class InitializerWrapper implements VertexStateUpdateListener, TaskStateUpdateListener {
 
 
     private final RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input;
@@ -242,15 +269,18 @@ public class RootInputInitializerManager {
     private final String vertexLogIdentifier;
     private final StateChangeNotifier stateChangeNotifier;
     private final List<String> notificationRegisteredVertices = Lists.newArrayList();
+    private final AppContext appContext;
 
     InitializerWrapper(RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input,
                        InputInitializer initializer, InputInitializerContext context,
-                       Vertex vertex, StateChangeNotifier stateChangeNotifier) {
+                       Vertex vertex, StateChangeNotifier stateChangeNotifier,
+                       AppContext appContext) {
       this.input = input;
       this.initializer = initializer;
       this.context = context;
       this.vertexLogIdentifier = vertex.getLogIdentifier();
       this.stateChangeNotifier = stateChangeNotifier;
+      this.appContext = appContext;
     }
 
     public RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> getInput() {
@@ -272,6 +302,7 @@ public class RootInputInitializerManager {
     public void setComplete() {
       this.isComplete.set(true);
       unregisterForVertexStatusUpdates();
+      unregisterForTaskStatusUpdates();
     }
 
     public void registerForVertexStateUpdates(String vertexName, Set<VertexState> stateSet) {
@@ -302,7 +333,118 @@ public class RootInputInitializerManager {
         initializer.onVertexStateUpdated(event);
       }
     }
-  }
 
+    private final Map<String, Map<Integer, Integer>> firstSuccessfulAttemptMap = new HashMap<String, Map<Integer, Integer>>();
+    private final ListMultimap<String, TezEvent> pendingEvents = LinkedListMultimap.create();
+    private final List<String> taskNotificationRegisteredVertices = Lists.newLinkedList();
+
+    @InterfaceAudience.Private
+    @VisibleForTesting
+    public Map<String, Map<Integer, Integer>> getFirstSuccessfulAttemptMap() {
+      return this.firstSuccessfulAttemptMap;
+    }
+
+    @InterfaceAudience.Private
+    @VisibleForTesting
+    public ListMultimap<String, TezEvent> getPendingEvents() {
+      return this.pendingEvents;
+    }
+
+    @Override
+    public void onTaskSucceeded(String vertexName, TezTaskID taskId, int attemptId) {
+      // Notifications will only start coming in after an event is received, which is when we register for notifications.
+      // TODO TEZ-1577. Get rid of this.
+      if (attemptId == -1) {
+        throw new TezUncheckedException(
+            "AttemptId is -1. This is likely caused by TEZ-1577; recovery not supported when InputInitializerEvents are used");
+      }
+      Map<Integer, Integer> vertexSuccessfulAttemptMap = firstSuccessfulAttemptMap.get(vertexName);
+      Integer successfulAttempt = vertexSuccessfulAttemptMap.get(taskId);
+      if (successfulAttempt == null) {
+        successfulAttempt = attemptId;
+        vertexSuccessfulAttemptMap.put(taskId.getId(), successfulAttempt);
+      }
+
+      // Run through all the pending events for this srcVertex to see if any of them need to be dispatched.
+      List<TezEvent> events = pendingEvents.get(vertexName);
+      if (events != null && !events.isEmpty()) {
+        List<InputInitializerEvent> toForwardEvents = new LinkedList<InputInitializerEvent>();
+        Iterator<TezEvent> eventIterator = events.iterator();
+        while (eventIterator.hasNext()) {
+          TezEvent tezEvent = eventIterator.next();
+          int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId();
+          int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId();
+          if (taskIndex == taskId.getId()) {
+            // Process only if there's a pending event for the specific succeeded task
+            if (taskAttemptIndex == successfulAttempt) {
+              toForwardEvents.add((InputInitializerEvent) tezEvent.getEvent());
+            }
+            eventIterator.remove();
+          }
+        }
+        sendEvents(toForwardEvents);
+      }
+    }
+
+    public void handleInputInitializerEvents(Collection<TezEvent> tezEvents) {
+      List<InputInitializerEvent> toForwardEvents = new LinkedList<InputInitializerEvent>();
+      for (TezEvent tezEvent : tezEvents) {
+        String srcVertexName = tezEvent.getSourceInfo().getTaskVertexName();
+        int taskIndex = tezEvent.getSourceInfo().getTaskAttemptID().getTaskID().getId();
+        int taskAttemptIndex = tezEvent.getSourceInfo().getTaskAttemptID().getId();
+
+        Map<Integer, Integer> vertexSuccessfulAttemptMap =
+            firstSuccessfulAttemptMap.get(srcVertexName);
+        if (vertexSuccessfulAttemptMap == null) {
+          vertexSuccessfulAttemptMap = new HashMap<Integer, Integer>();
+          firstSuccessfulAttemptMap.put(srcVertexName, vertexSuccessfulAttemptMap);
+          // Seen first time. Register for task updates
+          stateChangeNotifier.registerForTaskSuccessUpdates(srcVertexName, this);
+          taskNotificationRegisteredVertices.add(srcVertexName);
+        }
+
+        // Determine the successful attempt for the task
+        Integer successfulAttemptInteger = vertexSuccessfulAttemptMap.get(taskIndex);
+        if (successfulAttemptInteger == null) {
+          // Check immediately if this task has succeeded, in case the notification came in before the event
+          Vertex srcVertex = appContext.getCurrentDAG().getVertex(srcVertexName);
+          Task task = srcVertex.getTask(taskIndex);
+          if (task.getState() == TaskState.SUCCEEDED) {
+            successfulAttemptInteger = task.getSuccessfulAttempt().getID().getId();
+            vertexSuccessfulAttemptMap.put(taskIndex, successfulAttemptInteger);
+          }
+        }
+
+        if (successfulAttemptInteger == null) {
+          // Queue events and await a notification
+          pendingEvents.put(srcVertexName, tezEvent);
+        } else {
+          // Handle the event immediately.
+          if (taskAttemptIndex == successfulAttemptInteger) {
+            toForwardEvents.add((InputInitializerEvent) tezEvent.getEvent());
+          } // Otherwise the event can be dropped
+        }
+      }
+      sendEvents(toForwardEvents);
+    }
+
+    private void sendEvents(List<InputInitializerEvent> events) {
+      if (events != null && !events.isEmpty()) {
+        try {
+          initializer.handleInputInitializerEvent(events);
+        } catch (Exception e) {
+          throw new TezUncheckedException(
+              "Initializer for input: " + getInput().getName() + " on vertex: " + getVertexLogIdentifier() +
+                  " failed to process events", e);
+        }
+      }
+    }
+
+    private void unregisterForTaskStatusUpdates() {
+      for (String vertexName : taskNotificationRegisteredVertices) {
+        stateChangeNotifier.unregisterForTaskSuccessUpdates(vertexName, this);
+      }
+    }
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
index 558fc61..dc18e9b 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/StateChangeNotifier.java
@@ -33,6 +33,7 @@ import com.google.common.collect.SetMultimap;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.tez.dag.api.TezUncheckedException;
 import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.dag.records.TezVertexID;
 
 /**
@@ -55,13 +56,11 @@ public class StateChangeNotifier {
     this.lastKnowStatesMap = LinkedListMultimap.create();
   }
 
+  // -------------- VERTEX STATE CHANGE SECTION ---------------
   public void registerForVertexUpdates(String vertexName,
                                        Set<org.apache.tez.dag.api.event.VertexState> stateSet,
                                        VertexStateUpdateListener listener) {
-    Preconditions.checkNotNull(vertexName, "VertexName cannot be null");
-    Vertex vertex = dag.getVertex(vertexName);
-    Preconditions.checkNotNull(vertex, "Vertex does not exist: " + vertexName);
-    TezVertexID vertexId = vertex.getVertexId();
+    TezVertexID vertexId = validateAndGetVertexId(vertexName);
     writeLock.lock();
     // Read within the lock, to ensure a consistent view is seen.
     List<VertexStateUpdate> previousUpdates = lastKnowStatesMap.get(vertexId);
@@ -88,14 +87,8 @@ public class StateChangeNotifier {
     }
   }
 
-  // KKK Send out current state.
-
-
   public void unregisterForVertexUpdates(String vertexName, VertexStateUpdateListener listener) {
-    Preconditions.checkNotNull(vertexName, "VertexName cannot be null");
-    Vertex vertex = dag.getVertex(vertexName);
-    Preconditions.checkNotNull(vertex, "Vertex does not exist: " + vertexName);
-    TezVertexID vertexId = vertex.getVertexId();
+    TezVertexID vertexId = validateAndGetVertexId(vertexName);
     writeLock.lock();
     try {
       ListenerContainer listenerContainer = new ListenerContainer(listener, null);
@@ -125,6 +118,7 @@ public class StateChangeNotifier {
 
   }
 
+
   private static final class ListenerContainer {
     final VertexStateUpdateListener listener;
     final Set<org.apache.tez.dag.api.event.VertexState> states;
@@ -165,4 +159,64 @@ public class StateChangeNotifier {
       return System.identityHashCode(listener);
     }
   }
+
+  // -------------- END OF VERTEX STATE CHANGE SECTION ---------------
+
+  // -------------- TASK STATE CHANGE SECTION ---------------
+
+  // Task updates are not buffered to avoid storing unnecessary information.
+  // Components (non user facing) which use this will receive notifications after registration.
+  // They will have to query task states, prior to registration.
+  // Currently only handling Task SUCCESS events.
+  private final SetMultimap<TezVertexID, TaskStateUpdateListener> taskListeners =
+      Multimaps.synchronizedSetMultimap(HashMultimap.<TezVertexID, TaskStateUpdateListener>create());
+  private final ReentrantReadWriteLock taskListenerLock = new ReentrantReadWriteLock();
+  private final ReentrantReadWriteLock.ReadLock taskReadLock = taskListenerLock.readLock();
+  private final ReentrantReadWriteLock.WriteLock taskWriteLock = taskListenerLock.writeLock();
+
+
+
+  public void registerForTaskSuccessUpdates(String vertexName, TaskStateUpdateListener listener) {
+    TezVertexID vertexId = validateAndGetVertexId(vertexName);
+    Preconditions.checkNotNull(listener, "listener cannot be null");
+    taskWriteLock.lock();
+    try {
+      taskListeners.put(vertexId, listener);
+    } finally {
+      taskWriteLock.unlock();
+    }
+  }
+
+  public void unregisterForTaskSuccessUpdates(String vertexName, TaskStateUpdateListener listener) {
+    TezVertexID vertexId = validateAndGetVertexId(vertexName);
+    Preconditions.checkNotNull(listener, "listener cannot be null");
+    taskWriteLock.lock();
+    try {
+      taskListeners.remove(vertexId, listener);
+    } finally {
+      taskWriteLock.unlock();
+    }
+  }
+
+  public void taskSucceeded(String vertexName, TezTaskID taskId, int attemptId) {
+    taskReadLock.lock();
+    try {
+      for (TaskStateUpdateListener listener : taskListeners.get(taskId.getVertexID())) {
+        listener.onTaskSucceeded(vertexName, taskId, attemptId);
+      }
+    } finally {
+      taskReadLock.unlock();
+    }
+  }
+
+  // -------------- END OF TASK STATE CHANGE SECTION ---------------
+
+
+  private TezVertexID validateAndGetVertexId(String vertexName) {
+    Preconditions.checkNotNull(vertexName, "VertexName cannot be null");
+    Vertex vertex = dag.getVertex(vertexName);
+    Preconditions.checkNotNull(vertex, "Vertex does not exist: " + vertexName);
+    return vertex.getVertexId();
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.java
new file mode 100644
index 0000000..7c86991
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/TaskStateUpdateListener.java
@@ -0,0 +1,35 @@
+/*
+ * 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;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.tez.dag.api.event.VertexStateUpdate;
+import org.apache.tez.dag.api.oldrecords.TaskState;
+import org.apache.tez.dag.records.TezTaskID;
+
+@InterfaceAudience.Private
+/**
+ * This class should not be implemented by user facing APIs such as InputInitializer
+ */
+public interface TaskStateUpdateListener {
+
+  // Internal usage only. Currently only supporting onSuccess notifications for tasks.
+  // Exposing the taskID is ok, since this isn't public
+  public void onTaskSucceeded(String vertexName, TezTaskID taskId, int attemptId);
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 1dd711b..976f10f 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
@@ -29,6 +29,7 @@ import java.util.concurrent.locks.Lock;
 import java.util.concurrent.locks.ReadWriteLock;
 import java.util.concurrent.locks.ReentrantReadWriteLock;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.lang.StringUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
@@ -39,7 +40,6 @@ import org.apache.hadoop.yarn.event.EventHandler;
 import org.apache.hadoop.yarn.state.InvalidStateTransitonException;
 import org.apache.hadoop.yarn.state.MultipleArcTransition;
 import org.apache.hadoop.yarn.state.SingleArcTransition;
-import org.apache.hadoop.yarn.state.StateMachine;
 import org.apache.hadoop.yarn.state.StateMachineFactory;
 import org.apache.hadoop.yarn.util.Clock;
 import org.apache.tez.common.counters.TezCounters;
@@ -53,6 +53,7 @@ import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.StateChangeNotifier;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
@@ -88,6 +89,8 @@ import org.apache.tez.runtime.api.OutputCommitter;
 import org.apache.tez.runtime.api.impl.TezEvent;
 
 import com.google.common.annotations.VisibleForTesting;
+import org.apache.tez.state.OnStateChangedCallback;
+import org.apache.tez.state.StateMachineTez;
 
 /**
  * Implementation of Task interface.
@@ -118,6 +121,7 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   private final ContainerContext containerContext;
   @VisibleForTesting
   long scheduledTime;
+  final StateChangeNotifier stateChangeNotifier;
 
   private final List<TezEvent> tezEventsForTaskAttempts = new ArrayList<TezEvent>();
   private static final List<TezEvent> EMPTY_TASK_ATTEMPT_TEZ_EVENTS =
@@ -138,6 +142,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   // Recovery related flags
   boolean recoveryStartEventSeen = false;
 
+  private static final TaskStateChangedCallback STATE_CHANGED_CALLBACK = new TaskStateChangedCallback();
+
   private static final StateMachineFactory
                <TaskImpl, TaskStateInternal, TaskEventType, TaskEvent>
             stateMachineFactory
@@ -261,7 +267,13 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     // create the topology tables
     .installTopology();
 
-  private final StateMachine<TaskStateInternal, TaskEventType, TaskEvent>
+  private void augmentStateMachine() {
+    stateMachine
+        .registerStateEnteredCallback(TaskStateInternal.SUCCEEDED,
+            STATE_CHANGED_CALLBACK);
+  }
+
+  private final StateMachineTez<TaskStateInternal, TaskEventType, TaskEvent, TaskImpl>
     stateMachine;
 
   // TODO: Recovery
@@ -318,7 +330,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
       TaskAttemptListener taskAttemptListener,
       Clock clock, TaskHeartbeatHandler thh, AppContext appContext,
       boolean leafVertex, Resource resource,
-      ContainerContext containerContext) {
+      ContainerContext containerContext,
+      StateChangeNotifier stateChangeNotifier) {
     this.conf = conf;
     this.clock = clock;
     ReadWriteLock readWriteLock = new ReentrantReadWriteLock();
@@ -333,11 +346,14 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     this.taskHeartbeatHandler = thh;
     this.eventHandler = eventHandler;
     this.appContext = appContext;
+    this.stateChangeNotifier = stateChangeNotifier;
 
     this.leafVertex = leafVertex;
     this.taskResource = resource;
     this.containerContext = containerContext;
-    stateMachine = stateMachineFactory.make(this);
+    stateMachine = new StateMachineTez<TaskStateInternal, TaskEventType, TaskEvent, TaskImpl>(
+        stateMachineFactory.make(this), this);
+    augmentStateMachine();
   }
 
   @Override
@@ -1423,6 +1439,27 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
     }
   }
 
+  private static class TaskStateChangedCallback
+      implements OnStateChangedCallback<TaskStateInternal, TaskImpl> {
+
+    @Override
+    public void onStateChanged(TaskImpl task, TaskStateInternal taskStateInternal) {
+      // Only registered for SUCCEEDED notifications at the moment
+      Preconditions.checkState(taskStateInternal == TaskStateInternal.SUCCEEDED);
+      TaskAttempt successfulAttempt = task.getSuccessfulAttempt();
+      // TODO TEZ-1577.
+      // This is a horrible hack to get around recovery issues. Without this, recovery would fail
+      // for successful vertices.
+      // With this, recovery will end up failing for DAGs making use of InputInitializerEvents
+      int succesfulAttemptInt = -1;
+      if (successfulAttempt != null) {
+        succesfulAttemptInt = successfulAttempt.getID().getId();
+      }
+      task.stateChangeNotifier.taskSucceeded(task.getVertex().getName(), task.getTaskId(),
+          succesfulAttemptInt);
+    }
+  }
+
   private void killUnfinishedAttempt(TaskAttempt attempt, String logMsg) {
     if (commitAttempt != null && commitAttempt.equals(attempt)) {
       LOG.info("Removing commit attempt: " + commitAttempt);

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 6437e5b..594c651 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
@@ -127,7 +127,7 @@ import org.apache.tez.dag.app.dag.impl.DAGImpl.VertexGroupInfo;
 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.VertexDataMovementEventsGeneratedEvent;
+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;
@@ -244,6 +244,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
   private boolean vertexAlreadyInitialized = false;
 
+  @VisibleForTesting
+  final List<TezEvent> pendingInitializerEvents = new LinkedList<TezEvent>();
+
   protected static final
     StateMachineFactory<VertexImpl, VertexState, VertexEventType, VertexEvent>
        stateMachineFactory
@@ -261,6 +264,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
                 EnumSet.of(VertexState.NEW),
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
                   new NullEdgeInitializedTransition())
+          .addTransition(VertexState.NEW, VertexState.NEW,
+                VertexEventType.V_ROUTE_EVENT,
+                ROUTE_EVENT_TRANSITION)
+          .addTransition(VertexState.NEW,  VertexState.NEW,
+                VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED,
+                SOURCE_TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION)
           .addTransition
               (VertexState.NEW,
                   EnumSet.of(VertexState.NEW, VertexState.INITED,
@@ -1079,8 +1088,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         }
         return recoveredState;
       case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-        VertexDataMovementEventsGeneratedEvent vEvent =
-            (VertexDataMovementEventsGeneratedEvent) historyEvent;
+        VertexRecoverableEventsGeneratedEvent vEvent =
+            (VertexRecoverableEventsGeneratedEvent) historyEvent;
         this.recoveredEvents.addAll(vEvent.getTezEvents());
         return recoveredState;
       default:
@@ -1774,7 +1783,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               (this.targetVertices != null ?
                 this.targetVertices.isEmpty() : true),
               this.taskResource,
-              conContext);
+              conContext,
+              this.stateChangeNotifier);
       this.addTask(task);
       if(LOG.isDebugEnabled()) {
         LOG.debug("Created task for vertex " + logIdentifier + ": " +
@@ -2218,6 +2228,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               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());
@@ -2661,6 +2679,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               + vertex.logIdentifier + ". Starting root input initializers: "
               + vertex.inputsWithInitializers.size());
           vertex.rootInputInitializerManager.runInputInitializers(inputList);
+          // Send pending rootInputInitializerEvents
+          vertex.rootInputInitializerManager.handleInitializerEvents(vertex.pendingInitializerEvents);
+          vertex.pendingInitializerEvents.clear();
           return VertexState.INITIALIZING;
         } else {
           boolean hasOneToOneUninitedSource = false;
@@ -2706,6 +2727,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           // state. This is handled in RootInputInitializedTransition specially.
           vertex.initWaitsForRootInitializers = true;
           vertex.rootInputInitializerManager.runInputInitializers(inputList);
+          // Send pending rootInputInitializerEvents
+          vertex.rootInputInitializerManager.handleInitializerEvents(vertex.pendingInitializerEvents);
+          vertex.pendingInitializerEvents.clear();
           return VertexState.INITIALIZING;
         }
         if (!vertex.uninitializedEdges.isEmpty()) {
@@ -2795,6 +2819,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       if (vertex.numInitializedInputs == vertex.inputsWithInitializers.size()) {
         // All inputs initialized, shutdown the initializer.
         vertex.rootInputInitializerManager.shutdown();
+        vertex.rootInputInitializerManager = null;
       }
 
       // done. check if we need to do the initialization
@@ -3064,6 +3089,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       }
       if (vertex.rootInputInitializerManager != null) {
         vertex.rootInputInitializerManager.shutdown();
+        vertex.rootInputInitializerManager = null;
       }
       vertex.finished(VertexState.FAILED,
           VertexTerminationCause.ROOT_INPUT_INIT_FAILURE);
@@ -3102,6 +3128,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       super.transition(vertex, event);
       if (vertex.rootInputInitializerManager != null) {
         vertex.rootInputInitializerManager.shutdown();
+        vertex.rootInputInitializerManager = null;
       }
     }
   }
@@ -3146,17 +3173,19 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
             + " with state: " + completionEvent.getTaskAttemptState()
             + " vertexState: " + vertex.getState());
 
+
       if (TaskAttemptStateInternal.SUCCEEDED.equals(completionEvent
           .getTaskAttemptState())) {
         vertex.numSuccessSourceAttemptCompletions++;
+
         if (vertex.getState() == VertexState.RUNNING) {
+          // Inform the vertex manager about the source task completing.
           vertex.vertexManager.onSourceTaskCompleted(completionEvent
               .getTaskAttemptId().getTaskID());
         } else {
           vertex.pendingReportedSrcCompletions.add(completionEvent.getTaskAttemptId());
         }
       }
-
     }
   }
 
@@ -3349,7 +3378,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       if (vertex.getAppContext().isRecoveryEnabled()
           && !recovered
           && !tezEvents.isEmpty()) {
-        List<TezEvent> dataMovementEvents =
+        List<TezEvent> recoveryEvents =
             Lists.newArrayList();
         for (TezEvent tezEvent : tezEvents) {
           if (!isEventFromVertex(vertex, tezEvent.getSourceInfo())) {
@@ -3357,14 +3386,15 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           }
           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)) {
-            dataMovementEvents.add(tezEvent);
+            || tezEvent.getEventType().equals(EventType.ROOT_INPUT_DATA_INFORMATION_EVENT)
+            || tezEvent.getEventType().equals(EventType.ROOT_INPUT_INITIALIZER_EVENT)) {
+            recoveryEvents.add(tezEvent);
           }
         }
-        if (!dataMovementEvents.isEmpty()) {
-          VertexDataMovementEventsGeneratedEvent historyEvent =
-              new VertexDataMovementEventsGeneratedEvent(vertex.vertexId,
-                  dataMovementEvents);
+        if (!recoveryEvents.isEmpty()) {
+          VertexRecoverableEventsGeneratedEvent historyEvent =
+              new VertexRecoverableEventsGeneratedEvent(vertex.vertexId,
+                  recoveryEvents);
           vertex.appContext.getHistoryHandler().handle(
               new DAGHistoryEvent(vertex.getDAGId(), historyEvent));
         }
@@ -3431,6 +3461,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           break;
         case VERTEX_MANAGER_EVENT:
         {
+          // VM events on task success only can be changed as part of TEZ-1532
           VertexManagerEvent vmEvent = (VertexManagerEvent) tezEvent.getEvent();
           Vertex target = vertex.getDAG().getVertex(vmEvent.getTargetVertexName());
           Preconditions.checkArgument(target != null,
@@ -3449,9 +3480,25 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           InputInitializerEvent riEvent = (InputInitializerEvent) tezEvent.getEvent();
           Vertex target = vertex.getDAG().getVertex(riEvent.getTargetVertexName());
           Preconditions.checkArgument(target != null,
-              "Event sent to unkown vertex: " + riEvent.getTargetVertexName());
+              "Event sent to unknown vertex: " + riEvent.getTargetVertexName());
+          riEvent.setSourceVertexName(tezEvent.getSourceInfo().getTaskVertexName());
           if (target == vertex) {
-            vertex.rootInputInitializerManager.handleInitializerEvent(riEvent);
+            if (vertex.rootInputDescriptors == null ||
+                !vertex.rootInputDescriptors.containsKey(riEvent.getTargetInputName())) {
+              throw new TezUncheckedException(
+                  "InputInitializerEvent targeted at unknown initializer on vertex " +
+                      vertex.logIdentifier + ", Event=" + riEvent);
+            }
+            if (vertex.getState() == VertexState.NEW) {
+              vertex.pendingInitializerEvents.add(tezEvent);
+            } else  if (vertex.getState() == VertexState.INITIALIZING) {
+              vertex.rootInputInitializerManager.handleInitializerEvents(Collections.singletonList(tezEvent));
+            } else {
+              // Currently, INITED and subsequent states means Initializer complete / failure
+              if (LOG.isDebugEnabled()) {
+                LOG.debug("Dropping event" + tezEvent + " since state is not INITIALIZING in " + vertex.getLogIdentifier() + ", state=" + vertex.getState());
+              }
+            }
           } else {
             checkEventSourceMetadata(vertex, sourceMeta);
             vertex.eventHandler.handle(new VertexEventRouteEvent(target.getVertexId(),

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
deleted file mode 100644
index 7ae73be..0000000
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexDataMovementEventsGeneratedEvent.java
+++ /dev/null
@@ -1,214 +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.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-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.impl.EventMetaData;
-import org.apache.tez.runtime.api.impl.EventType;
-import org.apache.tez.runtime.api.impl.TezEvent;
-
-import com.google.common.collect.Lists;
-
-public class VertexDataMovementEventsGeneratedEvent implements HistoryEvent {
-
-  private static final Log LOG = LogFactory.getLog(
-      VertexDataMovementEventsGeneratedEvent.class);
-  private List<TezEvent> events;
-  private TezVertexID vertexID;
-
-  public VertexDataMovementEventsGeneratedEvent(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)
-              .contains(event.getEventType())) {
-        this.events.add(event);
-      }
-    }
-    if (events.isEmpty()) {
-      throw new RuntimeException("Invalid creation of VertexDataMovementEventsGeneratedEvent"
-        + ", no data movement/information events provided");
-    }
-  }
-
-  public VertexDataMovementEventsGeneratedEvent() {
-  }
-
-  @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()));
-        }
-        if (event.getSourceInfo() != null) {
-          evtBuilder.setSourceInfo(convertEventMetaDataToProto(event.getSourceInfo()));
-        }
-        if (event.getDestinationInfo() != null) {
-          evtBuilder.setDestinationInfo(convertEventMetaDataToProto(event.getDestinationInfo()));
-        }
-        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());
-      }
-      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);
-      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/b4580a7b/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
new file mode 100644
index 0000000..a9f1fd2
--- /dev/null
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexRecoverableEventsGeneratedEvent.java
@@ -0,0 +1,223 @@
+/**
+ * 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.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+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 Log LOG = LogFactory.getLog(
+      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()));
+        }
+        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);
+      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/b4580a7b/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 821612a..93f217f 100644
--- a/tez-dag/src/main/proto/HistoryEvents.proto
+++ b/tez-dag/src/main/proto/HistoryEvents.proto
@@ -180,6 +180,7 @@ message TezDataMovementEventProto {
   optional DataMovementEventProto data_movement_event = 3;
   optional CompositeEventProto composite_data_movement_event = 4;
   optional RootInputDataInformationEventProto root_input_data_information_event = 5;
+  optional RootInputInitializerEventProto input_initializer_event = 6;
 }
 
 message VertexDataMovementEventsGeneratedProto {

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 f53643f..63cd9c5 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
@@ -25,6 +25,9 @@ import static org.junit.Assert.fail;
 import static org.mockito.Mockito.RETURNS_DEEP_STUBS;
 import static org.mockito.Mockito.mock;
 import static org.mockito.Mockito.when;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.eq;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -53,6 +56,7 @@ import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.StateChangeNotifier;
 import org.apache.tez.dag.app.dag.TaskStateInternal;
 import org.apache.tez.dag.app.dag.TaskTerminationCause;
 import org.apache.tez.dag.app.dag.Vertex;
@@ -494,6 +498,8 @@ public class TestTaskImpl {
 
     // The task should now have succeeded
     assertTaskSucceededState();
+    verify(mockTask.stateChangeNotifier).taskSucceeded(any(String.class), eq(taskId),
+        eq(mockTask.getLastAttempt().getID().getId()));
 
     eventHandler.events.clear();
     // Now fail the attempt after it has succeeded
@@ -548,7 +554,7 @@ public class TestTaskImpl {
     private List<MockTaskAttemptImpl> taskAttempts = new LinkedList<MockTaskAttemptImpl>();
     private Vertex vertex;
     TaskLocationHint locationHint;
-    
+
     public MockTaskImpl(TezVertexID vertexId, int partition,
         EventHandler eventHandler, Configuration conf,
         TaskAttemptListener taskAttemptListener, Clock clock,
@@ -557,7 +563,7 @@ public class TestTaskImpl {
         ContainerContext containerContext, Vertex vertex) {
       super(vertexId, partition, eventHandler, conf, taskAttemptListener,
           clock, thh, appContext, leafVertex, resource,
-          containerContext);
+          containerContext, mock(StateChangeNotifier.class));
       this.vertex = vertex;
       this.locationHint = locationHint;
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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
index c5153b6..bd13ffe 100644
--- 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
@@ -47,6 +47,7 @@ import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
 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;
@@ -186,7 +187,7 @@ public class TestTaskRecovery {
             new Configuration(), mock(TaskAttemptListener.class),
             new SystemClock(), mock(TaskHeartbeatHandler.class),
             mockAppContext, false, Resource.newInstance(1, 1),
-            mock(ContainerContext.class));
+            mock(ContainerContext.class), mock(StateChangeNotifier.class));
 
     Map<String, OutputCommitter> committers =
         new HashMap<String, OutputCommitter>();


[23/25] git commit: TEZ-1533. Request Events more often if a complete set of events is received by a task. (sseth)

Posted by je...@apache.org.
TEZ-1533. Request Events more often if a complete set of events is
received by a task. (sseth)


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

Branch: refs/heads/TEZ-8
Commit: 8bd2ca352664fab8d248593f60faa6c0ff223920
Parents: 3b34c41
Author: Siddharth Seth <ss...@apache.org>
Authored: Tue Sep 16 03:54:45 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Tue Sep 16 03:54:45 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/tez/runtime/task/TaskReporter.java   |  53 ++++++---
 .../tez/runtime/task/TestTaskReporter.java      | 115 +++++++++++++++++++
 3 files changed, 151 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8bd2ca35/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ca4e7e2..83048bb 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -42,6 +42,7 @@ ALL CHANGES
   of DAG submission
   TEZ-1571. Add create method for DataSinkDescriptor.
   TEZ-1585. Memory leak in tez session mode.
+  TEZ-1533. Request Events more often if a complete set of events is received by a task.
 
 Release 0.5.0: 2014-09-03
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8bd2ca35/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java b/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
index fcb8778..15dcbb0 100644
--- a/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
+++ b/tez-dag/src/main/java/org/apache/tez/runtime/task/TaskReporter.java
@@ -115,7 +115,8 @@ public class TaskReporter {
     heartbeatExecutor.shutdownNow();
   }
 
-  private static class HeartbeatCallable implements Callable<Boolean> {
+  @VisibleForTesting
+  static class HeartbeatCallable implements Callable<Boolean> {
 
     private static final int LOG_COUNTER_START_INTERVAL = 5000; // 5 seconds
     private static final float LOG_COUNTER_BACKOFF = 1.3f;
@@ -172,25 +173,30 @@ public class TaskReporter {
     public Boolean call() throws Exception {
       // Heartbeat only for active tasks. Errors, etc will be reported directly.
       while (!task.isTaskDone() && !task.hadFatalError()) {
-        boolean result = heartbeat(null);
-        if (!result) {
+        ResponseWrapper response = heartbeat(null);
+
+        if (response.shouldDie) {
           // AM sent a shouldDie=true
           LOG.info("Asked to die via task heartbeat");
           return false;
-        }
-        lock.lock();
-        try {
-          boolean interrupted = condition.await(pollInterval, TimeUnit.MILLISECONDS);
-          if (!interrupted) {
-            nonOobHeartbeatCounter++;
+        } else {
+          if (response.numEvents < maxEventsToGet) {
+            // Wait before sending another heartbeat. Otherwise consider as an OOB heartbeat
+            lock.lock();
+            try {
+              boolean interrupted = condition.await(pollInterval, TimeUnit.MILLISECONDS);
+              if (!interrupted) {
+                nonOobHeartbeatCounter++;
+              }
+            } finally {
+              lock.unlock();
+            }
           }
-        } finally {
-          lock.unlock();
         }
       }
       int pendingEventCount = eventsToSend.size();
       if (pendingEventCount > 0) {
-        LOG.warn("Exiting TaskReporter therad with pending queue size=" + pendingEventCount);
+        LOG.warn("Exiting TaskReporter thread with pending queue size=" + pendingEventCount);
       }
       return true;
     }
@@ -203,7 +209,7 @@ public class TaskReporter {
      * @throws TezException
      *           indicates an exception somewhere in the AM.
      */
-    private synchronized boolean heartbeat(Collection<TezEvent> eventsArg) throws IOException,
+    private synchronized ResponseWrapper heartbeat(Collection<TezEvent> eventsArg) throws IOException,
         TezException {
 
       if (eventsArg != null) {
@@ -247,7 +253,7 @@ public class TaskReporter {
 
       if (response.shouldDie()) {
         LOG.info("Received should die response from AM");
-        return false;
+        return new ResponseWrapper(true, 1);
       }
       if (response.getLastRequestId() != requestId) {
         throw new TezException("AM and Task out of sync" + ", responseReqId="
@@ -256,6 +262,7 @@ public class TaskReporter {
 
       // The same umbilical is used by multiple tasks. Problematic in the case where multiple tasks
       // are running using the same umbilical.
+      int numEventsReceived = 0;
       if (task.isTaskDone() || task.hadFatalError()) {
         if (response.getEvents() != null && !response.getEvents().isEmpty()) {
           LOG.warn("Current task already complete, Ignoring all event in"
@@ -268,11 +275,11 @@ public class TaskReporter {
                 + task.getTaskAttemptID() + ", eventCount=" + response.getEvents().size());
           }
           // This should ideally happen in a separate thread
+          numEventsReceived = response.getEvents().size();
           task.handleEvents(response.getEvents());
         }
       }
-      return true;
-
+      return new ResponseWrapper(false, numEventsReceived);
     }
 
     public void markComplete() {
@@ -308,7 +315,7 @@ public class TaskReporter {
           task.getProgress()), updateEventMetadata);
       TezEvent taskCompletedEvent = new TezEvent(new TaskAttemptCompletedEvent(),
           updateEventMetadata);
-      return heartbeat(Lists.newArrayList(statusUpdateEvent, taskCompletedEvent));
+      return !heartbeat(Lists.newArrayList(statusUpdateEvent, taskCompletedEvent)).shouldDie;
     }
 
     /**
@@ -334,7 +341,7 @@ public class TaskReporter {
       }
       TezEvent taskAttemptFailedEvent = new TezEvent(new TaskAttemptFailedEvent(diagnostics),
           srcMeta == null ? updateEventMetadata : srcMeta);
-      return heartbeat(Lists.newArrayList(statusUpdateEvent, taskAttemptFailedEvent));
+      return !heartbeat(Lists.newArrayList(statusUpdateEvent, taskAttemptFailedEvent)).shouldDie;
     }
 
     private void addEvents(TezTaskAttemptID taskAttemptID, Collection<TezEvent> events) {
@@ -381,4 +388,14 @@ public class TaskReporter {
   public boolean canCommit(TezTaskAttemptID taskAttemptID) throws IOException {
     return umbilical.canCommit(taskAttemptID);
   }
+
+  private static final class ResponseWrapper {
+    boolean shouldDie;
+    int numEvents;
+
+    private ResponseWrapper(boolean shouldDie, int numEvents) {
+      this.shouldDie = shouldDie;
+      this.numEvents = numEvents;
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/8bd2ca35/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java b/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java
new file mode 100644
index 0000000..de03307
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/runtime/task/TestTaskReporter.java
@@ -0,0 +1,115 @@
+/*
+ * 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.runtime.task;
+
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.doAnswer;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.timeout;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import com.google.common.collect.Lists;
+import org.apache.tez.common.TezTaskUmbilicalProtocol;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.runtime.LogicalIOProcessorRuntimeTask;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.TezHeartbeatRequest;
+import org.apache.tez.runtime.api.impl.TezHeartbeatResponse;
+import org.junit.Test;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+public class TestTaskReporter {
+
+  @Test(timeout = 10000)
+  public void testContinuousHeartbeatsOnMaxEvents() throws Exception {
+
+    final Object lock = new Object();
+    final AtomicBoolean hb2Done = new AtomicBoolean(false);
+
+    TezTaskUmbilicalProtocol mockUmbilical = mock(TezTaskUmbilicalProtocol.class);
+    doAnswer(new Answer() {
+      @Override
+      public Object answer(InvocationOnMock invocation) throws Throwable {
+        Object[] args = invocation.getArguments();
+        TezHeartbeatRequest request = (TezHeartbeatRequest) args[0];
+        if (request.getRequestId() == 1 || request.getRequestId() == 2) {
+          TezHeartbeatResponse response = new TezHeartbeatResponse(createEvents(5));
+          response.setLastRequestId(request.getRequestId());
+          return response;
+        } else if (request.getRequestId() == 3) {
+          TezHeartbeatResponse response = new TezHeartbeatResponse(createEvents(1));
+          response.setLastRequestId(request.getRequestId());
+          synchronized (lock) {
+            hb2Done.set(true);
+            lock.notify();
+          }
+          return response;
+        } else {
+          throw new TezUncheckedException("Invalid request id for test: " + request.getRequestId());
+        }
+      }
+    }).when(mockUmbilical).heartbeat(any(TezHeartbeatRequest.class));
+
+    TezTaskAttemptID mockTaskAttemptId = mock(TezTaskAttemptID.class);
+    LogicalIOProcessorRuntimeTask mockTask = mock(LogicalIOProcessorRuntimeTask.class);
+    doReturn("vertexName").when(mockTask).getVertexName();
+    doReturn(mockTaskAttemptId).when(mockTask).getTaskAttemptID();
+
+    // Setup the sleep time to be way higher than the test timeout
+    TaskReporter.HeartbeatCallable heartbeatCallable =
+        new TaskReporter.HeartbeatCallable(mockTask, mockUmbilical, 100000, 100000, 5,
+            new AtomicLong(0),
+            "containerIdStr");
+
+    ExecutorService executor = Executors.newSingleThreadExecutor();
+    executor.submit(heartbeatCallable);
+    try {
+      synchronized (lock) {
+        if (!hb2Done.get()) {
+          lock.wait();
+        }
+      }
+      verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class));
+      Thread.sleep(2000l);
+      // Sleep for 2 seconds, less than the callable sleep time. No more invocations.
+      verify(mockUmbilical, times(3)).heartbeat(any(TezHeartbeatRequest.class));
+    } finally {
+      executor.shutdownNow();
+    }
+
+  }
+
+  private List<TezEvent> createEvents(int numEvents) {
+    List<TezEvent> list = Lists.newArrayListWithCapacity(numEvents);
+    for (int i = 0; i < numEvents; i++) {
+      list.add(mock(TezEvent.class));
+    }
+    return list;
+  }
+}


[14/25] git commit: TEZ-1578. Remove TeraSort from Tez codebase. (hitesh)

Posted by je...@apache.org.
TEZ-1578. Remove TeraSort from Tez codebase. (hitesh)


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

Branch: refs/heads/TEZ-8
Commit: 8e382b34d9487faea56ac0a8a31c53745aacdc3d
Parents: b62298e
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 11:11:47 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 11:11:47 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +
 .../tez/mapreduce/examples/ExampleDriver.java   |   9 -
 .../mapreduce/examples/terasort/GenSort.java    | 251 -------------
 .../mapreduce/examples/terasort/Random16.java   | 374 -------------------
 .../examples/terasort/TeraChecksum.java         | 103 -----
 .../mapreduce/examples/terasort/TeraGen.java    | 311 ---------------
 .../examples/terasort/TeraInputFormat.java      | 353 -----------------
 .../examples/terasort/TeraOutputFormat.java     | 112 ------
 .../examples/terasort/TeraScheduler.java        | 253 -------------
 .../mapreduce/examples/terasort/TeraSort.java   | 335 -----------------
 .../examples/terasort/TeraValidate.java         | 188 ----------
 .../mapreduce/examples/terasort/Unsigned16.java | 297 ---------------
 .../examples/terasort/TestTeraSort.java         |  94 -----
 13 files changed, 3 insertions(+), 2680 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index ff53e04..87729b3 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -5,6 +5,7 @@ Release 0.6.0: Unreleased
 
 INCOMPATIBLE CHANGES
   TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
+  TEZ-1578. Remove TeraSort from Tez codebase.
 
 ALL CHANGES:
   TEZ-1544. Link to release artifacts for 0.5.0 does not point to a specific link for 0.5.0.
@@ -12,6 +13,8 @@ ALL CHANGES:
   TEZ-850. Recovery unit tests.
   TEZ-853. Support counters recovery.
   TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized.
+  TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts.
+  TEZ-1578. Remove TeraSort from Tez codebase.
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
index 977b767..3824607 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/ExampleDriver.java
@@ -31,9 +31,6 @@ import org.apache.tez.dag.api.client.DAGStatus;
 import org.apache.tez.dag.api.client.Progress;
 import org.apache.tez.dag.api.client.StatusGetOpts;
 import org.apache.tez.dag.api.client.VertexStatus;
-import org.apache.tez.mapreduce.examples.terasort.TeraGen;
-import org.apache.tez.mapreduce.examples.terasort.TeraSort;
-import org.apache.tez.mapreduce.examples.terasort.TeraValidate;
 
 /**
  * A description of an example program based on its class and a
@@ -65,12 +62,6 @@ public class ExampleDriver {
       pgd.addClass("join", Join.class,
           "A job that effects a join over sorted, equally partitioned"
           + " datasets");
-      pgd.addClass("teragen", TeraGen.class,
-          "Generate data for the terasort");
-      pgd.addClass("terasort", TeraSort.class,
-          "Run the terasort");
-      pgd.addClass("teravalidate", TeraValidate.class,
-          "Checking results of terasort");
       pgd.addClass("groupbyorderbymrrtest", GroupByOrderByMRRTest.class,
           "A map-reduce-reduce program that does groupby-order by. Takes input"
           + " containing employee_name department name per line of input"

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
deleted file mode 100644
index c8517e5..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/GenSort.java
+++ /dev/null
@@ -1,251 +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.mapreduce.examples.terasort;
-
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.io.PrintStream;
-import java.math.BigInteger;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.util.PureJavaCrc32;
-
-/** 
- * A single process data generator for the terasort data. Based on gensort.c 
- * version 1.1 (3 Mar 2009) from Chris Nyberg <ch...@ordinal.com>.
- */
-public class GenSort {
-
-  /**
-   * Generate a "binary" record suitable for all sort benchmarks *except* 
-   * PennySort.
-   */
-  static void generateRecord(byte[] recBuf, Unsigned16 rand, 
-                                     Unsigned16 recordNumber) {
-    /* generate the 10-byte key using the high 10 bytes of the 128-bit
-     * random number
-     */
-    for(int i=0; i < 10; ++i) {
-      recBuf[i] = rand.getByte(i);
-    }
-
-    /* add 2 bytes of "break" */
-    recBuf[10] = 0x00;
-    recBuf[11] = 0x11;
-
-    /* convert the 128-bit record number to 32 bits of ascii hexadecimal
-     * as the next 32 bytes of the record.
-     */
-    for (int i = 0; i < 32; i++) {
-      recBuf[12 + i] = (byte) recordNumber.getHexDigit(i);
-    }
-
-    /* add 4 bytes of "break" data */
-    recBuf[44] = (byte) 0x88;
-    recBuf[45] = (byte) 0x99;
-    recBuf[46] = (byte) 0xAA;
-    recBuf[47] = (byte) 0xBB;
-
-    /* add 48 bytes of filler based on low 48 bits of random number */
-    for(int i=0; i < 12; ++i) {
-      recBuf[48+i*4] = recBuf[49+i*4] = recBuf[50+i*4] = recBuf[51+i*4] =
-        (byte) rand.getHexDigit(20 + i);
-    }
-
-    /* add 4 bytes of "break" data */
-    recBuf[96] = (byte) 0xCC;
-    recBuf[97] = (byte) 0xDD;
-    recBuf[98] = (byte) 0xEE;
-    recBuf[99] = (byte) 0xFF;
-  }
-
-
-  private static BigInteger makeBigInteger(long x) {
-    byte[] data = new byte[8];
-    for(int i=0; i < 8; ++i) {
-      data[i] = (byte) (x >>> (56 - 8*i));
-    }
-    return new BigInteger(1, data);
-  }
-
-  private static final BigInteger NINETY_FIVE = new BigInteger("95");
-
-  /**
-   * Generate an ascii record suitable for all sort benchmarks including 
-   * PennySort.
-   */
-  static void generateAsciiRecord(byte[] recBuf, Unsigned16 rand, 
-                                  Unsigned16 recordNumber) {
-
-    /* generate the 10-byte ascii key using mostly the high 64 bits.
-     */
-    long temp = rand.getHigh8();
-    if (temp < 0) {
-      // use biginteger to avoid the negative sign problem
-      BigInteger bigTemp = makeBigInteger(temp);
-      recBuf[0] = (byte) (' ' + (bigTemp.mod(NINETY_FIVE).longValue()));
-      temp = bigTemp.divide(NINETY_FIVE).longValue();
-    } else {
-      recBuf[0] = (byte) (' ' + (temp % 95));
-      temp /= 95;      
-    }
-    for(int i=1; i < 8; ++i) {
-      recBuf[i] = (byte) (' ' + (temp % 95));
-      temp /= 95;      
-    }
-    temp = rand.getLow8();
-    if (temp < 0) {
-      BigInteger bigTemp = makeBigInteger(temp);
-      recBuf[8] = (byte) (' ' + (bigTemp.mod(NINETY_FIVE).longValue()));
-      temp = bigTemp.divide(NINETY_FIVE).longValue();      
-    } else {
-      recBuf[8] = (byte) (' ' + (temp % 95));
-      temp /= 95;
-    }
-    recBuf[9] = (byte)(' ' + (temp % 95));
-
-    /* add 2 bytes of "break" */
-    recBuf[10] = ' ';
-    recBuf[11] = ' ';
-
-    /* convert the 128-bit record number to 32 bits of ascii hexadecimal
-     * as the next 32 bytes of the record.
-     */
-    for (int i = 0; i < 32; i++) {
-      recBuf[12 + i] = (byte) recordNumber.getHexDigit(i);
-    }
-
-    /* add 2 bytes of "break" data */
-    recBuf[44] = ' ';
-    recBuf[45] = ' ';
-
-    /* add 52 bytes of filler based on low 48 bits of random number */
-    for(int i=0; i < 13; ++i) {
-      recBuf[46+i*4] = recBuf[47+i*4] = recBuf[48+i*4] = recBuf[49+i*4] =
-        (byte) rand.getHexDigit(19 + i);
-    }
-
-    /* add 2 bytes of "break" data */
-    recBuf[98] = '\r';	/* nice for Windows */
-    recBuf[99] = '\n';
-}
-
-
-  private static void usage() {
-    PrintStream out = System.out;
-    out.println("usage: gensort [-a] [-c] [-bSTARTING_REC_NUM] NUM_RECS FILE_NAME");
-    out.println("-a        Generate ascii records required for PennySort or JouleSort.");
-    out.println("          These records are also an alternative input for the other");
-    out.println("          sort benchmarks.  Without this flag, binary records will be");
-    out.println("          generated that contain the highest density of randomness in");
-    out.println("          the 10-byte key.");
-    out.println( "-c        Calculate the sum of the crc32 checksums of each of the");
-    out.println("          generated records and send it to standard error.");
-    out.println("-bN       Set the beginning record generated to N. By default the");
-    out.println("          first record generated is record 0.");
-    out.println("NUM_RECS  The number of sequential records to generate.");
-    out.println("FILE_NAME The name of the file to write the records to.\n");
-    out.println("Example 1 - to generate 1000000 ascii records starting at record 0 to");
-    out.println("the file named \"pennyinput\":");
-    out.println("    gensort -a 1000000 pennyinput\n");
-    out.println("Example 2 - to generate 1000 binary records beginning with record 2000");
-    out.println("to the file named \"partition2\":");
-    out.println("    gensort -b2000 1000 partition2");
-    System.exit(1);
-  }
-
-
-  public static void outputRecords(OutputStream out,
-                                   boolean useAscii,
-                                   Unsigned16 firstRecordNumber,
-                                   Unsigned16 recordsToGenerate,
-                                   Unsigned16 checksum
-                                   ) throws IOException {
-    byte[] row = new byte[100];
-    Unsigned16 recordNumber = new Unsigned16(firstRecordNumber);
-    Unsigned16 lastRecordNumber = new Unsigned16(firstRecordNumber);
-    Checksum crc = new PureJavaCrc32();
-    Unsigned16 tmp = new Unsigned16();
-    lastRecordNumber.add(recordsToGenerate);
-    Unsigned16 ONE = new Unsigned16(1);
-    Unsigned16 rand = Random16.skipAhead(firstRecordNumber);
-    while (!recordNumber.equals(lastRecordNumber)) {
-      Random16.nextRand(rand);
-      if (useAscii) {
-        generateAsciiRecord(row, rand, recordNumber);
-      } else {
-        generateRecord(row, rand, recordNumber);
-      }
-      if (checksum != null) {
-        crc.reset();
-        crc.update(row, 0, row.length);
-        tmp.set(crc.getValue());
-        checksum.add(tmp);
-      }
-      recordNumber.add(ONE);
-      out.write(row);
-    }
-  }
-                                   
-  public static void main(String[] args) throws Exception {
-    Unsigned16 startingRecord = new Unsigned16();
-    Unsigned16 numberOfRecords;
-    OutputStream out;
-    boolean useAscii = false;
-    Unsigned16 checksum = null;
-
-    int i;
-    for(i=0; i < args.length; ++i) {
-      String arg = args[i];
-      int argLength = arg.length();
-      if (argLength >= 1 && arg.charAt(0) == '-') {
-        if (argLength < 2) {
-          usage();
-        }
-        switch (arg.charAt(1)) {
-        case 'a':
-          useAscii = true;
-          break;
-        case 'b':
-          startingRecord = Unsigned16.fromDecimal(arg.substring(2));
-          break;
-        case 'c':
-          checksum = new Unsigned16();
-          break;
-        default:
-          usage();
-        }
-      } else {
-        break;
-      }
-    }
-    if (args.length - i != 2) {
-      usage();
-    }
-    numberOfRecords = Unsigned16.fromDecimal(args[i]);
-    out = new FileOutputStream(args[i+1]);
-
-    outputRecords(out, useAscii, startingRecord, numberOfRecords, checksum);
-    out.close();
-    if (checksum != null) {
-      System.out.println(checksum);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
deleted file mode 100644
index 31cbd48..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Random16.java
+++ /dev/null
@@ -1,374 +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.mapreduce.examples.terasort;
-
-/**
- * This class implements a 128-bit linear congruential generator.
- * Specifically, if X0 is the most recently issued 128-bit random
- * number (or a seed of 0 if no random number has already been generated,
- * the next number to be generated, X1, is equal to:
- * X1 = (a * X0 + c) mod 2**128
- * where a is 47026247687942121848144207491837523525
- *            or 0x2360ed051fc65da44385df649fccf645
- *   and c is 98910279301475397889117759788405497857
- *            or 0x4a696d47726179524950202020202001
- * The coefficient "a" is suggested by:
- * Pierre L'Ecuyer, "Tables of linear congruential generators of different
- * sizes and good lattice structure", Mathematics of Computation, 68
- * pp. 249 - 260 (1999)
- * http://www.ams.org/mcom/1999-68-225/S0025-5718-99-00996-5/S0025-5718-99-00996-5.pdf
- * The constant "c" meets the simple suggestion by the same reference that
- * it be odd.
- *
- * There is also a facility for quickly advancing the state of the
- * generator by a fixed number of steps - this facilitates parallel
- * generation.
- *
- * This is based on 1.0 of rand16.c from Chris Nyberg 
- * <ch...@ordinal.com>.
- */
-class Random16 {
-
-  /** 
-   * The "Gen" array contain powers of 2 of the linear congruential generator.
-   * The index 0 struct contain the "a" coefficient and "c" constant for the
-   * generator.  That is, the generator is:
-   *    f(x) = (Gen[0].a * x + Gen[0].c) mod 2**128
-   *
-   * All structs after the first contain an "a" and "c" that
-   * comprise the square of the previous function.
-   *
-   * f**2(x) = (Gen[1].a * x + Gen[1].c) mod 2**128
-   * f**4(x) = (Gen[2].a * x + Gen[2].c) mod 2**128
-   * f**8(x) = (Gen[3].a * x + Gen[3].c) mod 2**128
-   * ...
-
-   */
-  private static class RandomConstant {
-    final Unsigned16 a;
-    final Unsigned16 c;
-    public RandomConstant(String left, String right) {
-      a = new Unsigned16(left);
-      c = new Unsigned16(right);
-    }
-  }
-
-  private static final RandomConstant[] genArray = new RandomConstant[]{
-    /* [  0] */ new RandomConstant("2360ed051fc65da44385df649fccf645", 
-                                   "4a696d47726179524950202020202001"),
-    /* [  1] */ new RandomConstant("17bce35bdf69743c529ed9eb20e0ae99", 
-                                   "95e0e48262b3edfe04479485c755b646"),
-    /* [  2] */ new RandomConstant("f4dd417327db7a9bd194dfbe42d45771", 
-                                   "882a02c315362b60765f100068b33a1c"),
-    /* [  3] */ new RandomConstant("6347af777a7898f6d1a2d6f33505ffe1", 
-                                   "5efc4abfaca23e8ca8edb1f2dfbf6478"),
-    /* [  4] */ new RandomConstant("b6a4239f3b315f84f6ef6d3d288c03c1", 
-                                   "f25bd15439d16af594c1b1bafa6239f0"),
-    /* [  5] */ new RandomConstant("2c82901ad1cb0cd182b631ba6b261781", 
-                                   "89ca67c29c9397d59c612596145db7e0"),
-    /* [  6] */ new RandomConstant("dab03f988288676ee49e66c4d2746f01", 
-                                   "8b6ae036713bd578a8093c8eae5c7fc0"),
-    /* [  7] */ new RandomConstant("602167331d86cf5684fe009a6d09de01", 
-                                   "98a2542fd23d0dbdff3b886cdb1d3f80"),
-    /* [  8] */ new RandomConstant("61ecb5c24d95b058f04c80a23697bc01", 
-                                   "954db923fdb7933e947cd1edcecb7f00"),
-    /* [  9] */ new RandomConstant("4a5c31e0654c28aa60474e83bf3f7801", 
-                                   "00be4a36657c98cd204e8c8af7dafe00"),
-    /* [ 10] */ new RandomConstant("ae4f079d54fbece1478331d3c6bef001", 
-                                   "991965329dccb28d581199ab18c5fc00"),
-    /* [ 11] */ new RandomConstant("101b8cb830c7cb927ff1ed50ae7de001", 
-                                   "e1a8705b63ad5b8cd6c3d268d5cbf800"),
-    /* [ 12] */ new RandomConstant("f54a27fc056b00e7563f3505e0fbc001", 
-                                   "2b657bbfd6ed9d632079e70c3c97f000"),
-    /* [ 13] */ new RandomConstant("df8a6fc1a833d201f98d719dd1f78001",
-                                   "59b60ee4c52fa49e9fe90682bd2fe000"),
-    /* [ 14] */ new RandomConstant("5480a5015f101a4ea7e3f183e3ef0001", 
-                                   "cc099c88030679464fe86aae8a5fc000"),
-    /* [ 15] */ new RandomConstant("a498509e76e5d7925f539c28c7de0001", 
-                                   "06b9abff9f9f33dd30362c0154bf8000"),
-    /* [ 16] */ new RandomConstant("0798a3d8b10dc72e60121cd58fbc0001", 
-                                   "e296707121688d5a0260b293a97f0000"),
-    /* [ 17] */ new RandomConstant("1647d1e78ec02e665fafcbbb1f780001", 
-                                   "189ffc4701ff23cb8f8acf6b52fe0000"),
-    /* [ 18] */ new RandomConstant("a7c982285e72bf8c0c8ddfb63ef00001", 
-                                   "5141110ab208fb9d61fb47e6a5fc0000"),
-    /* [ 19] */ new RandomConstant("3eb78ee8fb8c56dbc5d4e06c7de00001", 
-                                   "3c97caa62540f2948d8d340d4bf80000"),
-    /* [ 20] */ new RandomConstant("72d03b6f4681f2f9fe8e44d8fbc00001", 
-                                   "1b25cb9cfe5a0c963174f91a97f00000"),
-    /* [ 21] */ new RandomConstant("ea85f81e4f502c9bc8ae99b1f7800001", 
-                                   "0c644570b4a487103c5436352fe00000"),
-    /* [ 22] */ new RandomConstant("629c320db08b00c6bfa57363ef000001", 
-                                   "3d0589c28869472bde517c6a5fc00000"),
-    /* [ 23] */ new RandomConstant("c5c4b9ce268d074a386be6c7de000001", 
-                                   "bc95e5ab36477e65534738d4bf800000"),
-    /* [ 24] */ new RandomConstant("f30bbbbed1596187555bcd8fbc000001", 
-                                   "ddb02ff72a031c01011f71a97f000000"),
-    /* [ 25] */ new RandomConstant("4a1000fb26c9eeda3cc79b1f78000001", 
-                                   "2561426086d9acdb6c82e352fe000000"),
-    /* [ 26] */ new RandomConstant("89fb5307f6bf8ce2c1cf363ef0000001", 
-                                   "64a788e3c118ed1c8215c6a5fc000000"),
-    /* [ 27] */ new RandomConstant("830b7b3358a5d67ea49e6c7de0000001", 
-                                   "e65ea321908627cfa86b8d4bf8000000"),
-    /* [ 28] */ new RandomConstant("fd8a51da91a69fe1cd3cd8fbc0000001", 
-                                   "53d27225604d85f9e1d71a97f0000000"),
-    /* [ 29] */ new RandomConstant("901a48b642b90b55aa79b1f780000001", 
-                                   "ca5ec7a3ed1fe55e07ae352fe0000000"),
-    /* [ 30] */ new RandomConstant("118cdefdf32144f394f363ef00000001", 
-                                   "4daebb2e085330651f5c6a5fc0000000"),
-    /* [ 31] */ new RandomConstant("0a88c0a91cff430829e6c7de00000001", 
-                                   "9d6f1a00a8f3f76e7eb8d4bf80000000"),
-    /* [ 32] */ new RandomConstant("433bef4314f16a9453cd8fbc00000001", 
-                                   "158c62f2b31e496dfd71a97f00000000"),
-    /* [ 33] */ new RandomConstant("c294b02995ae6738a79b1f7800000001", 
-                                   "290e84a2eb15fd1ffae352fe00000000"),
-    /* [ 34] */ new RandomConstant("913575e0da8b16b14f363ef000000001", 
-                                   "e3dc1bfbe991a34ff5c6a5fc00000000"),
-    /* [ 35] */ new RandomConstant("2f61b9f871cf4e629e6c7de000000001", 
-                                   "ddf540d020b9eadfeb8d4bf800000000"),
-    /* [ 36] */ new RandomConstant("78d26ccbd68320c53cd8fbc000000001", 
-                                   "8ee4950177ce66bfd71a97f000000000"),
-    /* [ 37] */ new RandomConstant("8b7ebd037898518a79b1f78000000001", 
-                                   "39e0f787c907117fae352fe000000000"),
-    /* [ 38] */ new RandomConstant("0b5507b61f78e314f363ef0000000001", 
-                                   "659d2522f7b732ff5c6a5fc000000000"),
-    /* [ 39] */ new RandomConstant("4f884628f812c629e6c7de0000000001", 
-                                   "9e8722938612a5feb8d4bf8000000000"),
-    /* [ 40] */ new RandomConstant("be896744d4a98c53cd8fbc0000000001", 
-                                   "e941a65d66b64bfd71a97f0000000000"),
-    /* [ 41] */ new RandomConstant("daf63a553b6318a79b1f780000000001", 
-                                   "7b50d19437b097fae352fe0000000000"),
-    /* [ 42] */ new RandomConstant("2d7a23d8bf06314f363ef00000000001", 
-                                   "59d7b68e18712ff5c6a5fc0000000000"),
-    /* [ 43] */ new RandomConstant("392b046a9f0c629e6c7de00000000001", 
-                                   "4087bab2d5225feb8d4bf80000000000"),
-    /* [ 44] */ new RandomConstant("eb30fbb9c218c53cd8fbc00000000001", 
-                                   "b470abc03b44bfd71a97f00000000000"),
-    /* [ 45] */ new RandomConstant("b9cdc30594318a79b1f7800000000001", 
-                                   "366630eaba897fae352fe00000000000"),
-    /* [ 46] */ new RandomConstant("014ab453686314f363ef000000000001", 
-                                   "a2dfc77e8512ff5c6a5fc00000000000"),
-    /* [ 47] */ new RandomConstant("395221c7d0c629e6c7de000000000001", 
-                                   "1e0d25a14a25feb8d4bf800000000000"),
-    /* [ 48] */ new RandomConstant("4d972813a18c53cd8fbc000000000001", 
-                                   "9d50a5d3944bfd71a97f000000000000"),
-    /* [ 49] */ new RandomConstant("06f9e2374318a79b1f78000000000001", 
-                                   "bf7ab5eb2897fae352fe000000000000"),
-    /* [ 50] */ new RandomConstant("bd220cae86314f363ef0000000000001", 
-                                   "925b14e6512ff5c6a5fc000000000000"),
-    /* [ 51] */ new RandomConstant("36fd3a5d0c629e6c7de0000000000001", 
-                                   "724cce0ca25feb8d4bf8000000000000"),
-    /* [ 52] */ new RandomConstant("60def8ba18c53cd8fbc0000000000001", 
-                                   "1af42d1944bfd71a97f0000000000000"),
-    /* [ 53] */ new RandomConstant("8d500174318a79b1f780000000000001", 
-                                   "0f529e32897fae352fe0000000000000"),
-    /* [ 54] */ new RandomConstant("48e842e86314f363ef00000000000001", 
-                                   "844e4c6512ff5c6a5fc0000000000000"),
-    /* [ 55] */ new RandomConstant("4af185d0c629e6c7de00000000000001",
-                                   "9f40d8ca25feb8d4bf80000000000000"),
-    /* [ 56] */ new RandomConstant("7a670ba18c53cd8fbc00000000000001",
-                                   "9912b1944bfd71a97f00000000000000"),
-    /* [ 57] */ new RandomConstant("86de174318a79b1f7800000000000001",
-                                   "9c69632897fae352fe00000000000000"),
-    /* [ 58] */ new RandomConstant("55fc2e86314f363ef000000000000001",
-                                   "e1e2c6512ff5c6a5fc00000000000000"),
-    /* [ 59] */ new RandomConstant("ccf85d0c629e6c7de000000000000001",
-                                   "68058ca25feb8d4bf800000000000000"),
-    /* [ 60] */ new RandomConstant("1df0ba18c53cd8fbc000000000000001",
-                                   "610b1944bfd71a97f000000000000000"),
-    /* [ 61] */ new RandomConstant("4be174318a79b1f78000000000000001",
-                                   "061632897fae352fe000000000000000"),
-    /* [ 62] */ new RandomConstant("d7c2e86314f363ef0000000000000001",
-                                   "1c2c6512ff5c6a5fc000000000000000"),
-    /* [ 63] */ new RandomConstant("af85d0c629e6c7de0000000000000001",
-                                   "7858ca25feb8d4bf8000000000000000"),
-    /* [ 64] */ new RandomConstant("5f0ba18c53cd8fbc0000000000000001",
-                                   "f0b1944bfd71a97f0000000000000000"),
-    /* [ 65] */ new RandomConstant("be174318a79b1f780000000000000001",
-                                   "e1632897fae352fe0000000000000000"),
-    /* [ 66] */ new RandomConstant("7c2e86314f363ef00000000000000001",
-                                   "c2c6512ff5c6a5fc0000000000000000"),
-    /* [ 67] */ new RandomConstant("f85d0c629e6c7de00000000000000001",
-                                   "858ca25feb8d4bf80000000000000000"),
-    /* [ 68] */ new RandomConstant("f0ba18c53cd8fbc00000000000000001",
-                                   "0b1944bfd71a97f00000000000000000"),
-    /* [ 69] */ new RandomConstant("e174318a79b1f7800000000000000001",
-                                   "1632897fae352fe00000000000000000"),
-    /* [ 70] */ new RandomConstant("c2e86314f363ef000000000000000001",
-                                   "2c6512ff5c6a5fc00000000000000000"),
-    /* [ 71] */ new RandomConstant("85d0c629e6c7de000000000000000001",
-                                   "58ca25feb8d4bf800000000000000000"),
-    /* [ 72] */ new RandomConstant("0ba18c53cd8fbc000000000000000001",
-                                   "b1944bfd71a97f000000000000000000"),
-    /* [ 73] */ new RandomConstant("174318a79b1f78000000000000000001",
-                                   "632897fae352fe000000000000000000"),
-    /* [ 74] */ new RandomConstant("2e86314f363ef0000000000000000001",
-                                   "c6512ff5c6a5fc000000000000000000"),
-    /* [ 75] */ new RandomConstant("5d0c629e6c7de0000000000000000001",
-                                   "8ca25feb8d4bf8000000000000000000"),
-    /* [ 76] */ new RandomConstant("ba18c53cd8fbc0000000000000000001",
-                                   "1944bfd71a97f0000000000000000000"),
-    /* [ 77] */ new RandomConstant("74318a79b1f780000000000000000001",
-                                   "32897fae352fe0000000000000000000"),
-    /* [ 78] */ new RandomConstant("e86314f363ef00000000000000000001",
-                                   "6512ff5c6a5fc0000000000000000000"),
-    /* [ 79] */ new RandomConstant("d0c629e6c7de00000000000000000001",
-                                   "ca25feb8d4bf80000000000000000000"),
-    /* [ 80] */ new RandomConstant("a18c53cd8fbc00000000000000000001",
-                                   "944bfd71a97f00000000000000000000"),
-    /* [ 81] */ new RandomConstant("4318a79b1f7800000000000000000001",
-                                   "2897fae352fe00000000000000000000"),
-    /* [ 82] */ new RandomConstant("86314f363ef000000000000000000001",
-                                   "512ff5c6a5fc00000000000000000000"),
-    /* [ 83] */ new RandomConstant("0c629e6c7de000000000000000000001",
-                                   "a25feb8d4bf800000000000000000000"),
-    /* [ 84] */ new RandomConstant("18c53cd8fbc000000000000000000001",
-                                   "44bfd71a97f000000000000000000000"),
-    /* [ 85] */ new RandomConstant("318a79b1f78000000000000000000001",
-                                   "897fae352fe000000000000000000000"),
-    /* [ 86] */ new RandomConstant("6314f363ef0000000000000000000001",
-                                   "12ff5c6a5fc000000000000000000000"),
-    /* [ 87] */ new RandomConstant("c629e6c7de0000000000000000000001",
-                                   "25feb8d4bf8000000000000000000000"),
-    /* [ 88] */ new RandomConstant("8c53cd8fbc0000000000000000000001",
-                                   "4bfd71a97f0000000000000000000000"),
-    /* [ 89] */ new RandomConstant("18a79b1f780000000000000000000001",
-                                   "97fae352fe0000000000000000000000"),
-    /* [ 90] */ new RandomConstant("314f363ef00000000000000000000001",
-                                   "2ff5c6a5fc0000000000000000000000"),
-    /* [ 91] */ new RandomConstant("629e6c7de00000000000000000000001",
-                                   "5feb8d4bf80000000000000000000000"),
-    /* [ 92] */ new RandomConstant("c53cd8fbc00000000000000000000001",
-                                   "bfd71a97f00000000000000000000000"),
-    /* [ 93] */ new RandomConstant("8a79b1f7800000000000000000000001",
-                                   "7fae352fe00000000000000000000000"),
-    /* [ 94] */ new RandomConstant("14f363ef000000000000000000000001",
-                                   "ff5c6a5fc00000000000000000000000"),
-    /* [ 95] */ new RandomConstant("29e6c7de000000000000000000000001",
-                                   "feb8d4bf800000000000000000000000"),
-    /* [ 96] */ new RandomConstant("53cd8fbc000000000000000000000001",
-                                   "fd71a97f000000000000000000000000"),
-    /* [ 97] */ new RandomConstant("a79b1f78000000000000000000000001",
-                                   "fae352fe000000000000000000000000"),
-    /* [ 98] */ new RandomConstant("4f363ef0000000000000000000000001",
-                                   "f5c6a5fc000000000000000000000000"),
-    /* [ 99] */ new RandomConstant("9e6c7de0000000000000000000000001",
-                                   "eb8d4bf8000000000000000000000000"),
-    /* [100] */ new RandomConstant("3cd8fbc0000000000000000000000001",
-                                   "d71a97f0000000000000000000000000"),
-    /* [101] */ new RandomConstant("79b1f780000000000000000000000001",
-                                   "ae352fe0000000000000000000000000"),
-    /* [102] */ new RandomConstant("f363ef00000000000000000000000001",
-                                   "5c6a5fc0000000000000000000000000"),
-    /* [103] */ new RandomConstant("e6c7de00000000000000000000000001",
-                                   "b8d4bf80000000000000000000000000"),
-    /* [104] */ new RandomConstant("cd8fbc00000000000000000000000001",
-                                   "71a97f00000000000000000000000000"),
-    /* [105] */ new RandomConstant("9b1f7800000000000000000000000001",
-                                   "e352fe00000000000000000000000000"),
-    /* [106] */ new RandomConstant("363ef000000000000000000000000001",
-                                   "c6a5fc00000000000000000000000000"),
-    /* [107] */ new RandomConstant("6c7de000000000000000000000000001",
-                                   "8d4bf800000000000000000000000000"),
-    /* [108] */ new RandomConstant("d8fbc000000000000000000000000001",
-                                   "1a97f000000000000000000000000000"),
-    /* [109] */ new RandomConstant("b1f78000000000000000000000000001",
-                                   "352fe000000000000000000000000000"),
-    /* [110] */ new RandomConstant("63ef0000000000000000000000000001",
-                                   "6a5fc000000000000000000000000000"),
-    /* [111] */ new RandomConstant("c7de0000000000000000000000000001",
-                                   "d4bf8000000000000000000000000000"),
-    /* [112] */ new RandomConstant("8fbc0000000000000000000000000001",
-                                   "a97f0000000000000000000000000000"),
-    /* [113] */ new RandomConstant("1f780000000000000000000000000001",
-                                   "52fe0000000000000000000000000000"),
-    /* [114] */ new RandomConstant("3ef00000000000000000000000000001",
-                                   "a5fc0000000000000000000000000000"),
-    /* [115] */ new RandomConstant("7de00000000000000000000000000001",
-                                   "4bf80000000000000000000000000000"),
-    /* [116] */ new RandomConstant("fbc00000000000000000000000000001",
-                                   "97f00000000000000000000000000000"),
-    /* [117] */ new RandomConstant("f7800000000000000000000000000001",
-                                   "2fe00000000000000000000000000000"),
-    /* [118] */ new RandomConstant("ef000000000000000000000000000001",
-                                   "5fc00000000000000000000000000000"),
-    /* [119] */ new RandomConstant("de000000000000000000000000000001",
-                                   "bf800000000000000000000000000000"),
-    /* [120] */ new RandomConstant("bc000000000000000000000000000001",
-                                   "7f000000000000000000000000000000"),
-    /* [121] */ new RandomConstant("78000000000000000000000000000001",
-                                   "fe000000000000000000000000000000"),
-    /* [122] */ new RandomConstant("f0000000000000000000000000000001",
-                                   "fc000000000000000000000000000000"),
-    /* [123] */ new RandomConstant("e0000000000000000000000000000001",
-                                   "f8000000000000000000000000000000"),
-    /* [124] */ new RandomConstant("c0000000000000000000000000000001",
-                                   "f0000000000000000000000000000000"),
-    /* [125] */ new RandomConstant("80000000000000000000000000000001",
-                                   "e0000000000000000000000000000000"),
-    /* [126] */ new RandomConstant("00000000000000000000000000000001",
-                                   "c0000000000000000000000000000000"),
-    /* [127] */ new RandomConstant("00000000000000000000000000000001",
-                                   "80000000000000000000000000000000")};
-
-  /**
-   *  generate the random number that is "advance" steps
-   *  from an initial random number of 0.  This is done by
-   *  starting with 0, and then advancing the by the
-   *  appropriate powers of 2 of the linear congruential
-   *  generator.
-   */
-  public static Unsigned16 skipAhead(Unsigned16 advance) {
-    Unsigned16 result = new Unsigned16();
-    long          bit_map;
-
-    bit_map = advance.getLow8();
-    for (int i = 0; bit_map != 0 && i < 64; i++) {
-      if ((bit_map & (1L << i)) != 0) {
-        /* advance random number by f**(2**i) (x)
-         */
-        result.multiply(genArray[i].a);
-        result.add(genArray[i].c);
-        bit_map &= ~(1L << i);
-      }
-    }
-    bit_map = advance.getHigh8();
-    for (int i = 0; bit_map != 0 && i < 64; i++)
-    {
-      if ((bit_map & (1L << i)) != 0) {
-        /* advance random number by f**(2**(i + 64)) (x)
-         */
-        result.multiply(genArray[i+64].a);
-        result.add(genArray[i+64].c);
-        bit_map &= ~(1L << i);
-      }
-    }
-    return result;
-  }
-
-  /** 
-   * Generate the next 16 byte random number.
-   */
-  public static void nextRand(Unsigned16 rand) {
-    /* advance the random number forward once using the linear congruential
-     * generator, and then return the new random number
-     */
-    rand.multiply(genArray[0].a);
-    rand.add(genArray[0].c);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
deleted file mode 100644
index a5b408b..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraChecksum.java
+++ /dev/null
@@ -1,103 +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.mapreduce.examples.terasort;
-
-import java.io.IOException;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-public class TeraChecksum extends Configured implements Tool {
-  static class ChecksumMapper 
-      extends Mapper<Text, Text, NullWritable, Unsigned16> {
-    private Unsigned16 checksum = new Unsigned16();
-    private Unsigned16 sum = new Unsigned16();
-    private Checksum crc32 = new PureJavaCrc32();
-
-    public void map(Text key, Text value, 
-                    Context context) throws IOException {
-      crc32.reset();
-      crc32.update(key.getBytes(), 0, key.getLength());
-      crc32.update(value.getBytes(), 0, value.getLength());
-      checksum.set(crc32.getValue());
-      sum.add(checksum);
-    }
-
-    public void cleanup(Context context) 
-        throws IOException, InterruptedException {
-      context.write(NullWritable.get(), sum);
-    }
-  }
-
-  static class ChecksumReducer 
-      extends Reducer<NullWritable, Unsigned16, NullWritable, Unsigned16> {
-
-    public void reduce(NullWritable key, Iterable<Unsigned16> values,
-        Context context) throws IOException, InterruptedException  {
-      Unsigned16 sum = new Unsigned16();
-      for (Unsigned16 val : values) {
-        sum.add(val);
-      }
-      context.write(key, sum);
-    }
-  }
-
-  private static void usage() throws IOException {
-    System.err.println("terasum <out-dir> <report-dir>");
-  }
-
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    if (args.length != 2) {
-      usage();
-      return 2;
-    }
-    TeraInputFormat.setInputPaths(job, new Path(args[0]));
-    FileOutputFormat.setOutputPath(job, new Path(args[1]));
-    job.setJobName("TeraSum");
-    job.setJarByClass(TeraChecksum.class);
-    job.setMapperClass(ChecksumMapper.class);
-    job.setReducerClass(ChecksumReducer.class);
-    job.setOutputKeyClass(NullWritable.class);
-    job.setOutputValueClass(Unsigned16.class);
-    // force a single reducer
-    job.setNumReduceTasks(1);
-    job.setInputFormatClass(TeraInputFormat.class);
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  /**
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraChecksum(), args);
-    System.exit(res);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
deleted file mode 100644
index 70a4207..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraGen.java
+++ /dev/null
@@ -1,311 +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.mapreduce.examples.terasort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.zip.Checksum;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.LongWritable;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.io.WritableUtils;
-import org.apache.hadoop.mapreduce.Counter;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generate the official GraySort input data set.
- * The user specifies the number of rows and the output directory and this
- * class runs a map/reduce program to generate the data.
- * The format of the data is:
- * <ul>
- * <li>(10 bytes key) (constant 2 bytes) (32 bytes rowid) 
- *     (constant 4 bytes) (48 bytes filler) (constant 4 bytes)
- * <li>The rowid is the right justified row id as a hex number.
- * </ul>
- *
- * <p>
- * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar teragen 10000000000 in-dir</b>
- */
-public class TeraGen extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(TeraSort.class);
-
-  public static enum Counters {CHECKSUM}
-
-  public static final String NUM_ROWS = "mapreduce.terasort.num-rows";
-  /**
-   * An input format that assigns ranges of longs to each mapper.
-   */
-  static class RangeInputFormat 
-      extends InputFormat<LongWritable, NullWritable> {
-    
-    /**
-     * An input split consisting of a range on numbers.
-     */
-    static class RangeInputSplit extends InputSplit implements Writable {
-      long firstRow;
-      long rowCount;
-
-      public RangeInputSplit() { }
-
-      public RangeInputSplit(long offset, long length) {
-        firstRow = offset;
-        rowCount = length;
-      }
-
-      public long getLength() throws IOException {
-        return 0;
-      }
-
-      public String[] getLocations() throws IOException {
-        return new String[]{};
-      }
-
-      public void readFields(DataInput in) throws IOException {
-        firstRow = WritableUtils.readVLong(in);
-        rowCount = WritableUtils.readVLong(in);
-      }
-
-      public void write(DataOutput out) throws IOException {
-        WritableUtils.writeVLong(out, firstRow);
-        WritableUtils.writeVLong(out, rowCount);
-      }
-    }
-    
-    /**
-     * A record reader that will generate a range of numbers.
-     */
-    static class RangeRecordReader 
-        extends RecordReader<LongWritable, NullWritable> {
-      long startRow;
-      long finishedRows;
-      long totalRows;
-      LongWritable key = null;
-
-      public RangeRecordReader() {
-      }
-      
-      public void initialize(InputSplit split, TaskAttemptContext context) 
-          throws IOException, InterruptedException {
-        startRow = ((RangeInputSplit)split).firstRow;
-        finishedRows = 0;
-        totalRows = ((RangeInputSplit)split).rowCount;
-      }
-
-      public void close() throws IOException {
-        // NOTHING
-      }
-
-      public LongWritable getCurrentKey() {
-        return key;
-      }
-
-      public NullWritable getCurrentValue() {
-        return NullWritable.get();
-      }
-
-      public float getProgress() throws IOException {
-        return finishedRows / (float) totalRows;
-      }
-
-      public boolean nextKeyValue() {
-        if (key == null) {
-          key = new LongWritable();
-        }
-        if (finishedRows < totalRows) {
-          key.set(startRow + finishedRows);
-          finishedRows += 1;
-          return true;
-        } else {
-          return false;
-        }
-      }
-      
-    }
-
-    public RecordReader<LongWritable, NullWritable> 
-        createRecordReader(InputSplit split, TaskAttemptContext context) 
-        throws IOException {
-      return new RangeRecordReader();
-    }
-
-    /**
-     * Create the desired number of splits, dividing the number of rows
-     * between the mappers.
-     */
-    public List<InputSplit> getSplits(JobContext job) {
-      long totalRows = getNumberOfRows(job);
-      int numSplits = job.getConfiguration().getInt(MRJobConfig.NUM_MAPS, 1);
-      LOG.info("Generating " + totalRows + " using " + numSplits);
-      List<InputSplit> splits = new ArrayList<InputSplit>();
-      long currentRow = 0;
-      for(int split = 0; split < numSplits; ++split) {
-        long goal = 
-          (long) Math.ceil(totalRows * (double)(split + 1) / numSplits);
-        splits.add(new RangeInputSplit(currentRow, goal - currentRow));
-        currentRow = goal;
-      }
-      return splits;
-    }
-
-  }
-  
-  static long getNumberOfRows(JobContext job) {
-    return job.getConfiguration().getLong(NUM_ROWS, 0);
-  }
-  
-  static void setNumberOfRows(Job job, long numRows) {
-    job.getConfiguration().setLong(NUM_ROWS, numRows);
-  }
-
-  /**
-   * The Mapper class that given a row number, will generate the appropriate 
-   * output line.
-   */
-  public static class SortGenMapper 
-      extends Mapper<LongWritable, NullWritable, Text, Text> {
-
-    private Text key = new Text();
-    private Text value = new Text();
-    private Unsigned16 rand = null;
-    private Unsigned16 rowId = null;
-    private Unsigned16 checksum = new Unsigned16();
-    private Checksum crc32 = new PureJavaCrc32();
-    private Unsigned16 total = new Unsigned16();
-    private static final Unsigned16 ONE = new Unsigned16(1);
-    private byte[] buffer = new byte[TeraInputFormat.KEY_LENGTH +
-                                     TeraInputFormat.VALUE_LENGTH];
-    private Counter checksumCounter;
-
-    public void map(LongWritable row, NullWritable ignored,
-        Context context) throws IOException, InterruptedException {
-      if (rand == null) {
-        rowId = new Unsigned16(row.get());
-        rand = Random16.skipAhead(rowId);
-        checksumCounter = context.getCounter(Counters.CHECKSUM);
-      }
-      Random16.nextRand(rand);
-      GenSort.generateRecord(buffer, rand, rowId);
-      key.set(buffer, 0, TeraInputFormat.KEY_LENGTH);
-      value.set(buffer, TeraInputFormat.KEY_LENGTH, 
-                TeraInputFormat.VALUE_LENGTH);
-      context.write(key, value);
-      crc32.reset();
-      crc32.update(buffer, 0, 
-                   TeraInputFormat.KEY_LENGTH + TeraInputFormat.VALUE_LENGTH);
-      checksum.set(crc32.getValue());
-      total.add(checksum);
-      rowId.add(ONE);
-    }
-
-    @Override
-    public void cleanup(Context context) {
-      if (checksumCounter != null) {
-        checksumCounter.increment(total.getLow8());
-      }
-    }
-  }
-
-  private static void usage() throws IOException {
-    System.err.println("teragen <num rows> <output dir>");
-  }
-
-  /**
-   * Parse a number that optionally has a postfix that denotes a base.
-   * @param str an string integer with an option base {k,m,b,t}.
-   * @return the expanded value
-   */
-  private static long parseHumanLong(String str) {
-    char tail = str.charAt(str.length() - 1);
-    long base = 1;
-    switch (tail) {
-    case 't':
-      base *= 1000 * 1000 * 1000 * 1000;
-      break;
-    case 'b':
-      base *= 1000 * 1000 * 1000;
-      break;
-    case 'm':
-      base *= 1000 * 1000;
-      break;
-    case 'k':
-      base *= 1000;
-      break;
-    default:
-    }
-    if (base != 1) {
-      str = str.substring(0, str.length() - 1);
-    }
-    return Long.parseLong(str) * base;
-  }
-  
-  /**
-   * @param args the cli arguments
-   */
-  public int run(String[] args) 
-      throws IOException, InterruptedException, ClassNotFoundException {
-    Job job = Job.getInstance(getConf());
-    if (args.length != 2) {
-      usage();
-      return 2;
-    }
-    setNumberOfRows(job, parseHumanLong(args[0]));
-    Path outputDir = new Path(args[1]);
-    if (outputDir.getFileSystem(getConf()).exists(outputDir)) {
-      throw new IOException("Output directory " + outputDir + 
-                            " already exists.");
-    }
-    FileOutputFormat.setOutputPath(job, outputDir);
-    job.setJobName("TeraGen");
-    job.setJarByClass(TeraGen.class);
-    job.setMapperClass(SortGenMapper.class);
-    job.setNumReduceTasks(0);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setInputFormatClass(RangeInputFormat.class);
-    job.setOutputFormatClass(TeraOutputFormat.class);
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraGen(), args);
-    System.exit(res);
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
deleted file mode 100644
index 6c9b201..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraInputFormat.java
+++ /dev/null
@@ -1,353 +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.mapreduce.examples.terasort;
-
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.apache.commons.lang.exception.ExceptionUtils;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.TaskAttemptID;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl;
-import org.apache.hadoop.util.IndexedSortable;
-import org.apache.hadoop.util.QuickSort;
-
-/**
- * An input format that reads the first 10 characters of each line as the key
- * and the rest of the line as the value. Both key and value are represented
- * as Text.
- */
-public class TeraInputFormat extends FileInputFormat<Text,Text> {
-
-  static final String PARTITION_FILENAME = "_partition.lst";
-  private static final String NUM_PARTITIONS = 
-    "mapreduce.terasort.num.partitions";
-  private static final String SAMPLE_SIZE = 
-    "mapreduce.terasort.partitions.sample";
-  static final int KEY_LENGTH = 10;
-  static final int VALUE_LENGTH = 90;
-  static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
-  private static MRJobConfig lastContext = null;
-  private static List<InputSplit> lastResult = null;
-
-  static class TeraFileSplit extends FileSplit {
-    static private String[] ZERO_LOCATIONS = new String[0];
-
-    private String[] locations;
-
-    public TeraFileSplit() {
-      locations = ZERO_LOCATIONS;
-    }
-    public TeraFileSplit(Path file, long start, long length, String[] hosts) {
-      super(file, start, length, hosts);
-      try {
-        locations = super.getLocations();
-      } catch (IOException e) {
-        locations = ZERO_LOCATIONS;
-      }
-    }
-
-    // XXXXXX should this also be null-protected?
-    protected void setLocations(String[] hosts) {
-      locations = hosts;
-    }
-
-    @Override
-    public String[] getLocations() {
-      return locations;
-    }
-
-    public String toString() {
-      StringBuffer result = new StringBuffer();
-      result.append(getPath());
-      result.append(" from ");
-      result.append(getStart());
-      result.append(" length ");
-      result.append(getLength());
-      for(String host: getLocations()) {
-        result.append(" ");
-        result.append(host);
-      }
-      return result.toString();
-    }
-  }
-
-  static class TextSampler implements IndexedSortable {
-    private ArrayList<Text> records = new ArrayList<Text>();
-
-    public int compare(int i, int j) {
-      Text left = records.get(i);
-      Text right = records.get(j);
-      return left.compareTo(right);
-    }
-
-    public void swap(int i, int j) {
-      Text left = records.get(i);
-      Text right = records.get(j);
-      records.set(j, left);
-      records.set(i, right);
-    }
-
-    public void addKey(Text key) {
-      synchronized (this) {
-        records.add(new Text(key));
-      }
-    }
-
-    /**
-     * Find the split points for a given sample. The sample keys are sorted
-     * and down sampled to find even split points for the partitions. The
-     * returned keys should be the start of their respective partitions.
-     * @param numPartitions the desired number of partitions
-     * @return an array of size numPartitions - 1 that holds the split points
-     */
-    Text[] createPartitions(int numPartitions) {
-      int numRecords = records.size();
-      System.out.println("Making " + numPartitions + " from " + numRecords + 
-                         " sampled records");
-      if (numPartitions > numRecords) {
-        throw new IllegalArgumentException
-          ("Requested more partitions than input keys (" + numPartitions +
-           " > " + numRecords + ")");
-      }
-      new QuickSort().sort(this, 0, records.size());
-      float stepSize = numRecords / (float) numPartitions;
-      Text[] result = new Text[numPartitions-1];
-      for(int i=1; i < numPartitions; ++i) {
-        result[i-1] = records.get(Math.round(stepSize * i));
-      }
-      return result;
-    }
-  }
-  
-  /**
-   * Use the input splits to take samples of the input and generate sample
-   * keys. By default reads 100,000 keys from 10 locations in the input, sorts
-   * them and picks N-1 keys to generate N equally sized partitions.
-   * @param job the job to sample
-   * @param partFile where to write the output file to
-   * @throws Throwable if something goes wrong
-   */
-  public static void writePartitionFile(final JobContext job, 
-      Path partFile) throws Throwable  {
-    long t1 = System.currentTimeMillis();
-    Configuration conf = job.getConfiguration();
-    final TeraInputFormat inFormat = new TeraInputFormat();
-    final TextSampler sampler = new TextSampler();
-    int partitions = job.getNumReduceTasks();
-    long sampleSize = conf.getLong(SAMPLE_SIZE, 100000);
-    final List<InputSplit> splits = inFormat.getSplits(job);
-    long t2 = System.currentTimeMillis();
-    System.out.println("Computing input splits took " + (t2 - t1) + "ms");
-    int samples = Math.min(conf.getInt(NUM_PARTITIONS, 10), splits.size());
-    System.out.println("Sampling " + samples + " splits of " + splits.size());
-    final long recordsPerSample = sampleSize / samples;
-    final int sampleStep = splits.size() / samples;
-    Thread[] samplerReader = new Thread[samples];
-    SamplerThreadGroup threadGroup = new SamplerThreadGroup("Sampler Reader Thread Group");
-    // take N samples from different parts of the input
-    for(int i=0; i < samples; ++i) {
-      final int idx = i;
-      samplerReader[i] = 
-        new Thread (threadGroup,"Sampler Reader " + idx) {
-        {
-          setDaemon(true);
-        }
-        public void run() {
-          long records = 0;
-          try {
-            TaskAttemptContext context = new TaskAttemptContextImpl(
-              job.getConfiguration(), new TaskAttemptID());
-            RecordReader<Text, Text> reader = 
-              inFormat.createRecordReader(splits.get(sampleStep * idx),
-              context);
-            reader.initialize(splits.get(sampleStep * idx), context);
-            while (reader.nextKeyValue()) {
-              sampler.addKey(new Text(reader.getCurrentKey()));
-              records += 1;
-              if (recordsPerSample <= records) {
-                break;
-              }
-            }
-          } catch (IOException ie){
-            System.err.println("Got an exception while reading splits " +
-                ExceptionUtils.getStackTrace(ie));
-            throw new RuntimeException(ie);
-          } catch (InterruptedException e) {
-        	  
-          }
-        }
-      };
-      samplerReader[i].start();
-    }
-    FileSystem outFs = partFile.getFileSystem(conf);
-    DataOutputStream writer = outFs.create(partFile, true, 64*1024, (short) 10, 
-                                           outFs.getDefaultBlockSize(partFile));
-    for (int i = 0; i < samples; i++) {
-      try {
-        samplerReader[i].join();
-        if(threadGroup.getThrowable() != null){
-          throw threadGroup.getThrowable();
-        }
-      } catch (InterruptedException e) {
-      }
-    }
-    for(Text split : sampler.createPartitions(partitions)) {
-      split.write(writer);
-    }
-    writer.close();
-    long t3 = System.currentTimeMillis();
-    System.out.println("Computing parititions took " + (t3 - t2) + "ms");
-  }
-  
-  static class SamplerThreadGroup extends ThreadGroup{
-
-    private Throwable throwable;
-
-    public SamplerThreadGroup(String s) {
-      super(s);
-    }
-    
-    @Override
-    public void uncaughtException(Thread thread, Throwable throwable) {
-      this.throwable = throwable;
-    }
-    
-    public Throwable getThrowable() {
-      return this.throwable;
-    }
-    
-  }
-
-  static class TeraRecordReader extends RecordReader<Text,Text> {
-    private FSDataInputStream in;
-    private long offset;
-    private long length;
-    private static final int RECORD_LENGTH = KEY_LENGTH + VALUE_LENGTH;
-    private byte[] buffer = new byte[RECORD_LENGTH];
-    private Text key;
-    private Text value;
-
-    public TeraRecordReader() throws IOException {
-    }
-
-    public void initialize(InputSplit split, TaskAttemptContext context) 
-        throws IOException, InterruptedException {
-      Path p = ((FileSplit)split).getPath();
-      FileSystem fs = p.getFileSystem(context.getConfiguration());
-      in = fs.open(p);
-      long start = ((FileSplit)split).getStart();
-      // find the offset to start at a record boundary
-      offset = (RECORD_LENGTH - (start % RECORD_LENGTH)) % RECORD_LENGTH;
-      in.seek(start + offset);
-      length = ((FileSplit)split).getLength();
-    }
-
-    public void close() throws IOException {
-      in.close();
-    }
-
-    public Text getCurrentKey() {
-      return key;
-    }
-
-    public Text getCurrentValue() {
-      return value;
-    }
-
-    public float getProgress() throws IOException {
-      return (float) offset / length;
-    }
-
-    public boolean nextKeyValue() throws IOException {
-      if (offset >= length) {
-        return false;
-      }
-      int read = 0;
-      while (read < RECORD_LENGTH) {
-        long newRead = in.read(buffer, read, RECORD_LENGTH - read);
-        if (newRead == -1) {
-          if (read == 0) {
-            return false;
-          } else {
-            throw new EOFException("read past eof");
-          }
-        }
-        read += newRead;
-      }
-      if (key == null) {
-        key = new Text();
-      }
-      if (value == null) {
-        value = new Text();
-      }
-      key.set(buffer, 0, KEY_LENGTH);
-      value.set(buffer, KEY_LENGTH, VALUE_LENGTH);
-      offset += RECORD_LENGTH;
-      return true;
-    }
-  }
-
-  @Override
-  public RecordReader<Text, Text> 
-      createRecordReader(InputSplit split, TaskAttemptContext context) 
-      throws IOException {
-    return new TeraRecordReader();
-  }
-
-  protected FileSplit makeSplit(Path file, long start, long length, 
-                                String[] hosts) {
-    return new TeraFileSplit(file, start, length, hosts);
-  }
-
-  @Override
-  public List<InputSplit> getSplits(JobContext job) throws IOException {
-    if (job == lastContext) {
-      return lastResult;
-    }
-    long t1, t2, t3;
-    t1 = System.currentTimeMillis();
-    lastContext = job;
-    lastResult = super.getSplits(job);
-    t2 = System.currentTimeMillis();
-    System.out.println("Spent " + (t2 - t1) + "ms computing base-splits.");
-    if (job.getConfiguration().getBoolean(TeraScheduler.USE, true)) {
-      TeraScheduler scheduler = new TeraScheduler(
-        lastResult.toArray(new TeraFileSplit[0]), job.getConfiguration());
-      lastResult = scheduler.getNewFileSplits();
-      t3 = System.currentTimeMillis(); 
-      System.out.println("Spent " + (t3 - t2) + "ms computing TeraScheduler splits.");
-    }
-    return lastResult;
-  }
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
deleted file mode 100644
index 4ff0f06..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraOutputFormat.java
+++ /dev/null
@@ -1,112 +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.mapreduce.examples.terasort;
-
-import java.io.IOException;
-
-import org.apache.hadoop.fs.FSDataOutputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapred.InvalidJobConfException;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.OutputCommitter;
-import org.apache.hadoop.mapreduce.RecordWriter;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.mapreduce.security.TokenCache;
-
-/**
- * An output format that writes the key and value appended together.
- */
-public class TeraOutputFormat extends FileOutputFormat<Text,Text> {
-  static final String FINAL_SYNC_ATTRIBUTE = "mapreduce.terasort.final.sync";
-  private OutputCommitter committer = null;
-
-  /**
-   * Set the requirement for a final sync before the stream is closed.
-   */
-  static void setFinalSync(JobContext job, boolean newValue) {
-    job.getConfiguration().setBoolean(FINAL_SYNC_ATTRIBUTE, newValue);
-  }
-
-  /**
-   * Does the user want a final sync at close?
-   */
-  public static boolean getFinalSync(JobContext job) {
-    return job.getConfiguration().getBoolean(FINAL_SYNC_ATTRIBUTE, false);
-  }
-
-  static class TeraRecordWriter extends RecordWriter<Text,Text> {
-    private boolean finalSync = false;
-    private FSDataOutputStream out;
-
-    public TeraRecordWriter(FSDataOutputStream out,
-                            JobContext job) {
-      finalSync = getFinalSync(job);
-      this.out = out;
-    }
-
-    public synchronized void write(Text key, 
-                                   Text value) throws IOException {
-      out.write(key.getBytes(), 0, key.getLength());
-      out.write(value.getBytes(), 0, value.getLength());
-    }
-    
-    public void close(TaskAttemptContext context) throws IOException {
-      if (finalSync) {
-        out.hsync();
-      }
-      out.close();
-    }
-  }
-
-  @Override
-  public void checkOutputSpecs(JobContext job
-                              ) throws InvalidJobConfException, IOException {
-    // Ensure that the output directory is set
-    Path outDir = getOutputPath(job);
-    if (outDir == null) {
-      throw new InvalidJobConfException("Output directory not set in JobConf.");
-    }
-
-    // get delegation token for outDir's file system
-    TokenCache.obtainTokensForNamenodes(job.getCredentials(),
-        new Path[] { outDir }, job.getConfiguration());
-  }
-
-  public RecordWriter<Text,Text> getRecordWriter(TaskAttemptContext job
-                                                 ) throws IOException {
-    Path file = getDefaultWorkFile(job, "");
-    FileSystem fs = file.getFileSystem(job.getConfiguration());
-     FSDataOutputStream fileOut = fs.create(file);
-    return new TeraRecordWriter(fileOut, job);
-  }
-  
-  public OutputCommitter getOutputCommitter(TaskAttemptContext context) 
-      throws IOException {
-    if (committer == null) {
-      Path output = getOutputPath(context);
-      committer = new FileOutputCommitter(output, context);
-    }
-    return committer;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
deleted file mode 100644
index 82638d8..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraScheduler.java
+++ /dev/null
@@ -1,253 +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.mapreduce.examples.terasort;
-
-import java.io.*;
-import java.util.*;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.server.tasktracker.TTConfig;
-import org.apache.tez.mapreduce.examples.terasort.TeraInputFormat.TeraFileSplit;
-
-import com.google.common.base.Charsets;
-
-class TeraScheduler {
-  static String USE = "mapreduce.terasort.use.terascheduler";
-  private static final Log LOG = LogFactory.getLog(TeraScheduler.class);
-  private Split[] splits;
-  private List<Host> hosts = new ArrayList<Host>();
-  private int slotsPerHost;
-  private int remainingSplits = 0;
-  private FileSplit[] realSplits = null;
-
-  static class Split {
-    String filename;
-    boolean isAssigned = false;
-    List<Host> locations = new ArrayList<Host>();
-    Split(String filename) {
-      this.filename = filename;
-    }
-    public String toString() {
-      StringBuffer result = new StringBuffer();
-      result.append(filename);
-      result.append(" on ");
-      for(Host host: locations) {
-        result.append(host.hostname);
-        result.append(", ");
-      }
-      return result.toString();
-    }
-  }
-  static class Host {
-    String hostname;
-    List<Split> splits = new ArrayList<Split>();
-    Host(String hostname) {
-      this.hostname = hostname;
-    }
-    public String toString() {
-      StringBuffer result = new StringBuffer();
-      result.append(splits.size());
-      result.append(" ");
-      result.append(hostname);
-      return result.toString();
-    }
-  }
-
-  List<String> readFile(String filename) throws IOException {
-    List<String> result = new ArrayList<String>(10000);
-    BufferedReader in = new BufferedReader(
-        new InputStreamReader(new FileInputStream(filename), Charsets.UTF_8));
-    String line = in.readLine();
-    while (line != null) {
-      result.add(line);
-      line = in.readLine();
-    }
-    in.close();
-    return result;
-  }
-
-  public TeraScheduler(String splitFilename, 
-                       String nodeFilename) throws IOException {
-    slotsPerHost = 4;
-    // get the hosts
-    Map<String, Host> hostIds = new HashMap<String,Host>();
-    for(String hostName: readFile(nodeFilename)) {
-      Host host = new Host(hostName);
-      hosts.add(host);
-      hostIds.put(hostName, host);
-    }
-    // read the blocks
-    List<String> splitLines = readFile(splitFilename);
-    splits = new Split[splitLines.size()];
-    remainingSplits = 0;
-    for(String line: splitLines) {
-      StringTokenizer itr = new StringTokenizer(line);
-      Split newSplit = new Split(itr.nextToken());
-      splits[remainingSplits++] = newSplit;
-      while (itr.hasMoreTokens()) {
-        Host host = hostIds.get(itr.nextToken());
-        newSplit.locations.add(host);
-        host.splits.add(newSplit);
-      }
-    }
-  }
-
-  public TeraScheduler(FileSplit[] realSplits,
-                       Configuration conf) throws IOException {
-    this.realSplits = realSplits;
-    this.slotsPerHost = conf.getInt(TTConfig.TT_MAP_SLOTS, 4);
-    Map<String, Host> hostTable = new HashMap<String, Host>();
-    splits = new Split[realSplits.length];
-    for(FileSplit realSplit: realSplits) {
-      Split split = new Split(realSplit.getPath().toString());
-      splits[remainingSplits++] = split;
-      for(String hostname: realSplit.getLocations()) {
-        Host host = hostTable.get(hostname);
-        if (host == null) {
-          host = new Host(hostname);
-          hostTable.put(hostname, host);
-          hosts.add(host);
-        }
-        host.splits.add(split);
-        split.locations.add(host);
-      }
-    }
-  }
-
-  Host pickBestHost() {
-    Host result = null;
-    int splits = Integer.MAX_VALUE;
-    for(Host host: hosts) {
-      if (host.splits.size() < splits) {
-        result = host;
-        splits = host.splits.size();
-      }
-    }
-    if (result != null) {
-      hosts.remove(result);
-      LOG.debug("picking " + result);
-    }
-    return result;
-  }
-
-  void pickBestSplits(Host host) {
-    int tasksToPick = Math.min(slotsPerHost, 
-                               (int) Math.ceil((double) remainingSplits / 
-                                               hosts.size()));
-    Split[] best = new Split[tasksToPick];
-    for(Split cur: host.splits) {
-      LOG.debug("  examine: " + cur.filename + " " + cur.locations.size());
-      int i = 0;
-      while (i < tasksToPick && best[i] != null && 
-             best[i].locations.size() <= cur.locations.size()) {
-        i += 1;
-      }
-      if (i < tasksToPick) {
-        for(int j = tasksToPick - 1; j > i; --j) {
-          best[j] = best[j-1];
-        }
-        best[i] = cur;
-      }
-    }
-    // for the chosen blocks, remove them from the other locations
-    for(int i=0; i < tasksToPick; ++i) {
-      if (best[i] != null) {
-        LOG.debug(" best: " + best[i].filename);
-        for (Host other: best[i].locations) {
-          other.splits.remove(best[i]);
-        }
-        best[i].locations.clear();
-        best[i].locations.add(host);
-        best[i].isAssigned = true;
-        remainingSplits -= 1;
-      }
-    }
-    // for the non-chosen blocks, remove this host
-    for(Split cur: host.splits) {
-      if (!cur.isAssigned) {
-        cur.locations.remove(host);
-      }
-    }
-  }
-  
-  void solve() throws IOException {
-    Host host = pickBestHost();
-    while (host != null) {
-      pickBestSplits(host);
-      host = pickBestHost();
-    }
-  }
-
-  /**
-   * Solve the schedule and modify the FileSplit array to reflect the new
-   * schedule. It will move placed splits to front and unplacable splits
-   * to the end.
-   * @return a new list of FileSplits that are modified to have the
-   *    best host as the only host.
-   * @throws IOException
-   */
-  public List<InputSplit> getNewFileSplits() throws IOException {
-    solve();
-    FileSplit[] result = new FileSplit[realSplits.length];
-    int left = 0;
-    int right = realSplits.length - 1;
-    for(int i=0; i < splits.length; ++i) {
-      if (splits[i].isAssigned) {
-        // copy the split and fix up the locations
-        ((TeraFileSplit) realSplits[i]).setLocations
-           (new String[]{splits[i].locations.get(0).hostname});
-        result[left++] = realSplits[i];
-      } else {
-        result[right--] = realSplits[i];
-      }
-    }
-    List<InputSplit> ret = new ArrayList<InputSplit>();
-    for (FileSplit fs : result) {
-      ret.add(fs);
-    }
-    return ret;
-  }
-
-  public static void main(String[] args) throws IOException {
-    TeraScheduler problem = new TeraScheduler("block-loc.txt", "nodes");
-    for(Host host: problem.hosts) {
-      System.out.println(host);
-    }
-    LOG.info("starting solve");
-    problem.solve();
-    List<Split> leftOvers = new ArrayList<Split>();
-    for(int i=0; i < problem.splits.length; ++i) {
-      if (problem.splits[i].isAssigned) {
-        System.out.println("sched: " + problem.splits[i]);        
-      } else {
-        leftOvers.add(problem.splits[i]);
-      }
-    }
-    for(Split cur: leftOvers) {
-      System.out.println("left: " + cur);
-    }
-    System.out.println("left over: " + leftOvers.size());
-    LOG.info("done");
-  }
-
-}


[10/25] git commit: Fix accidental change to CHANGES.txt

Posted by je...@apache.org.
Fix accidental change to CHANGES.txt


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

Branch: refs/heads/TEZ-8
Commit: e709d62aaf56115a771a361bc83d9d4bdd7bb924
Parents: b4580a7
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 12 03:16:06 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 12 03:16:06 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt | 1 -
 1 file changed, 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e709d62a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index dd20f43..4abc9bf 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,7 +17,6 @@ Release 0.5.1: Unreleased
 
 INCOMPATIBLE CHANGES
   TEZ-1539. Change InputInitializerEvent semantics to SEND_ONCE_ON_TASK_SUCCESS
-  TEZ-1488. Rename HashComparator to ProxyComparator and implement in TezBytesComparator
 
 ALL CHANGES
   TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput()


[13/25] TEZ-1578. Remove TeraSort from Tez codebase. (hitesh)

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
deleted file mode 100644
index 5a097f2..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraSort.java
+++ /dev/null
@@ -1,335 +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.mapreduce.examples.terasort;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.PrintStream;
-import java.net.URI;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.MRJobConfig;
-import org.apache.hadoop.mapreduce.Partitioner;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generates the sampled split points, launches the job, and waits for it to
- * finish. 
- * <p>
- * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar terasort in-dir out-dir</b>
- */
-public class TeraSort extends Configured implements Tool {
-  private static final Log LOG = LogFactory.getLog(TeraSort.class);
-  static String SIMPLE_PARTITIONER = "mapreduce.terasort.simplepartitioner";
-  static String OUTPUT_REPLICATION = "mapreduce.terasort.output.replication";
-
-  /**
-   * A partitioner that splits text keys into roughly equal partitions
-   * in a global sorted order.
-   */
-  static class TotalOrderPartitioner extends Partitioner<Text,Text>
-      implements Configurable {
-    private TrieNode trie;
-    private Text[] splitPoints;
-    private Configuration conf;
-
-    /**
-     * A generic trie node
-     */
-    static abstract class TrieNode {
-      private int level;
-      TrieNode(int level) {
-        this.level = level;
-      }
-      abstract int findPartition(Text key);
-      abstract void print(PrintStream strm) throws IOException;
-      int getLevel() {
-        return level;
-      }
-    }
-
-    /**
-     * An inner trie node that contains 256 children based on the next
-     * character.
-     */
-    static class InnerTrieNode extends TrieNode {
-      private TrieNode[] child = new TrieNode[256];
-      
-      InnerTrieNode(int level) {
-        super(level);
-      }
-      int findPartition(Text key) {
-        int level = getLevel();
-        if (key.getLength() <= level) {
-          return child[0].findPartition(key);
-        }
-        return child[key.getBytes()[level] & 0xff].findPartition(key);
-      }
-      void setChild(int idx, TrieNode child) {
-        this.child[idx] = child;
-      }
-      void print(PrintStream strm) throws IOException {
-        for(int ch=0; ch < 256; ++ch) {
-          for(int i = 0; i < 2*getLevel(); ++i) {
-            strm.print(' ');
-          }
-          strm.print(ch);
-          strm.println(" ->");
-          if (child[ch] != null) {
-            child[ch].print(strm);
-          }
-        }
-      }
-    }
-
-    /**
-     * A leaf trie node that does string compares to figure out where the given
-     * key belongs between lower..upper.
-     */
-    static class LeafTrieNode extends TrieNode {
-      int lower;
-      int upper;
-      Text[] splitPoints;
-      LeafTrieNode(int level, Text[] splitPoints, int lower, int upper) {
-        super(level);
-        this.splitPoints = splitPoints;
-        this.lower = lower;
-        this.upper = upper;
-      }
-      int findPartition(Text key) {
-        for(int i=lower; i<upper; ++i) {
-          if (splitPoints[i].compareTo(key) > 0) {
-            return i;
-          }
-        }
-        return upper;
-      }
-      void print(PrintStream strm) throws IOException {
-        for(int i = 0; i < 2*getLevel(); ++i) {
-          strm.print(' ');
-        }
-        strm.print(lower);
-        strm.print(", ");
-        strm.println(upper);
-      }
-    }
-
-
-    /**
-     * Read the cut points from the given sequence file.
-     * @param fs the file system
-     * @param p the path to read
-     * @param job the job config
-     * @return the strings to split the partitions on
-     * @throws IOException
-     */
-    private static Text[] readPartitions(FileSystem fs, Path p,
-        Configuration conf) throws IOException {
-      int reduces = conf.getInt(MRJobConfig.NUM_REDUCES, 1);
-      Text[] result = new Text[reduces - 1];
-      DataInputStream reader = fs.open(p);
-      for(int i=0; i < reduces - 1; ++i) {
-        result[i] = new Text();
-        result[i].readFields(reader);
-      }
-      reader.close();
-      return result;
-    }
-
-    /**
-     * Given a sorted set of cut points, build a trie that will find the correct
-     * partition quickly.
-     * @param splits the list of cut points
-     * @param lower the lower bound of partitions 0..numPartitions-1
-     * @param upper the upper bound of partitions 0..numPartitions-1
-     * @param prefix the prefix that we have already checked against
-     * @param maxDepth the maximum depth we will build a trie for
-     * @return the trie node that will divide the splits correctly
-     */
-    private static TrieNode buildTrie(Text[] splits, int lower, int upper, 
-                                      Text prefix, int maxDepth) {
-      int depth = prefix.getLength();
-      if (depth >= maxDepth || lower == upper) {
-        return new LeafTrieNode(depth, splits, lower, upper);
-      }
-      InnerTrieNode result = new InnerTrieNode(depth);
-      Text trial = new Text(prefix);
-      // append an extra byte on to the prefix
-      trial.append(new byte[1], 0, 1);
-      int currentBound = lower;
-      for(int ch = 0; ch < 255; ++ch) {
-        trial.getBytes()[depth] = (byte) (ch + 1);
-        lower = currentBound;
-        while (currentBound < upper) {
-          if (splits[currentBound].compareTo(trial) >= 0) {
-            break;
-          }
-          currentBound += 1;
-        }
-        trial.getBytes()[depth] = (byte) ch;
-        result.child[ch] = buildTrie(splits, lower, currentBound, trial, 
-                                     maxDepth);
-      }
-      // pick up the rest
-      trial.getBytes()[depth] = (byte) 255;
-      result.child[255] = buildTrie(splits, currentBound, upper, trial,
-                                    maxDepth);
-      return result;
-    }
-
-    public void setConf(Configuration conf) {
-      try {
-        FileSystem fs = FileSystem.getLocal(conf);
-        this.conf = conf;
-        Path partFile = new Path(TeraInputFormat.PARTITION_FILENAME);
-        splitPoints = readPartitions(fs, partFile, conf);
-        trie = buildTrie(splitPoints, 0, splitPoints.length, new Text(), 2);
-      } catch (IOException ie) {
-        throw new IllegalArgumentException("can't read paritions file", ie);
-      }
-    }
-
-    public Configuration getConf() {
-      return conf;
-    }
-    
-    public TotalOrderPartitioner() {
-    }
-
-    public int getPartition(Text key, Text value, int numPartitions) {
-      return trie.findPartition(key);
-    }
-    
-  }
-  
-  /**
-   * A total order partitioner that assigns keys based on their first 
-   * PREFIX_LENGTH bytes, assuming a flat distribution.
-   */
-  public static class SimplePartitioner extends Partitioner<Text, Text>
-      implements Configurable {
-    int prefixesPerReduce;
-    private static final int PREFIX_LENGTH = 3;
-    private Configuration conf = null;
-    public void setConf(Configuration conf) {
-      this.conf = conf;
-      prefixesPerReduce = (int) Math.ceil((1 << (8 * PREFIX_LENGTH)) / 
-        (float) conf.getInt(MRJobConfig.NUM_REDUCES, 1));
-    }
-    
-    public Configuration getConf() {
-      return conf;
-    }
-    
-    @Override
-    public int getPartition(Text key, Text value, int numPartitions) {
-      byte[] bytes = key.getBytes();
-      int len = Math.min(PREFIX_LENGTH, key.getLength());
-      int prefix = 0;
-      for(int i=0; i < len; ++i) {
-        prefix = (prefix << 8) | (0xff & bytes[i]);
-      }
-      return prefix / prefixesPerReduce;
-    }
-  }
-
-  public static boolean getUseSimplePartitioner(JobContext job) {
-    return job.getConfiguration().getBoolean(SIMPLE_PARTITIONER, false);
-  }
-
-  public static void setUseSimplePartitioner(Job job, boolean value) {
-    job.getConfiguration().setBoolean(SIMPLE_PARTITIONER, value);
-  }
-
-  public static int getOutputReplication(JobContext job) {
-    return job.getConfiguration().getInt(OUTPUT_REPLICATION, 1);
-  }
-
-  public static void setOutputReplication(Job job, int value) {
-    job.getConfiguration().setInt(OUTPUT_REPLICATION, value);
-  }
-
-  public int run(String[] args) throws Exception {
-    if (args.length != 2) {
-      System.err.println("Invalid no. of arguments provided");
-      System.err.println("Usage: terasort <input-dir> <output-dir>");
-      return -1;
-    }
-
-    LOG.info("starting");
-    Job job = Job.getInstance(getConf());
-    Path inputDir = new Path(args[0]);
-    Path outputDir = new Path(args[1]);
-    boolean useSimplePartitioner = getUseSimplePartitioner(job);
-    TeraInputFormat.setInputPaths(job, inputDir);
-    FileOutputFormat.setOutputPath(job, outputDir);
-    job.setJobName("TeraSort");
-    job.setJarByClass(TeraSort.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    job.setInputFormatClass(TeraInputFormat.class);
-    job.setOutputFormatClass(TeraOutputFormat.class);
-    if (useSimplePartitioner) {
-      job.setPartitionerClass(SimplePartitioner.class);
-    } else {
-      long start = System.currentTimeMillis();
-      Path partitionFile = new Path(outputDir, 
-                                    TeraInputFormat.PARTITION_FILENAME);
-      URI partitionUri = new URI(partitionFile.toString() +
-                                 "#" + TeraInputFormat.PARTITION_FILENAME);
-      try {
-        TeraInputFormat.writePartitionFile(job, partitionFile);
-      } catch (Throwable e) {
-        LOG.error(e.getMessage());
-        return -1;
-      }
-      job.addCacheFile(partitionUri);  
-      long end = System.currentTimeMillis();
-      System.out.println("Spent " + (end - start) + "ms computing partitions.");
-      job.setPartitionerClass(TotalOrderPartitioner.class);
-    }
-    
-    job.getConfiguration().setInt("dfs.replication", getOutputReplication(job));
-    TeraOutputFormat.setFinalSync(job, true);
-    int ret = job.waitForCompletion(true) ? 0 : 1;
-    LOG.info("done");
-    return ret;
-  }
-
-  /**
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraSort(), args);
-    System.exit(res);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
deleted file mode 100644
index 24c3145..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/TeraValidate.java
+++ /dev/null
@@ -1,188 +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.mapreduce.examples.terasort;
-
-import java.io.IOException;
-import java.util.zip.Checksum;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.conf.Configured;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.io.BytesWritable;
-import org.apache.hadoop.io.Text;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Mapper;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.input.FileInputFormat;
-import org.apache.hadoop.mapreduce.lib.input.FileSplit;
-import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat;
-import org.apache.hadoop.util.PureJavaCrc32;
-import org.apache.hadoop.util.Tool;
-import org.apache.hadoop.util.ToolRunner;
-
-/**
- * Generate 1 mapper per a file that checks to make sure the keys
- * are sorted within each file. The mapper also generates 
- * "$file:begin", first key and "$file:end", last key. The reduce verifies that
- * all of the start/end items are in order.
- * Any output from the reduce is problem report.
- * <p>
- * To run the program: 
- * <b>bin/hadoop jar hadoop-*-examples.jar teravalidate out-dir report-dir</b>
- * <p>
- * If there is any output, something is wrong and the output of the reduce
- * will have the problem report.
- */
-public class TeraValidate extends Configured implements Tool {
-  private static final Text ERROR = new Text("error");
-  private static final Text CHECKSUM = new Text("checksum");
-  
-  private static String textifyBytes(Text t) {
-    BytesWritable b = new BytesWritable();
-    b.set(t.getBytes(), 0, t.getLength());
-    return b.toString();
-  }
-
-  static class ValidateMapper extends Mapper<Text,Text,Text,Text> {
-    private Text lastKey;
-    private String filename;
-    private Unsigned16 checksum = new Unsigned16();
-    private Unsigned16 tmp = new Unsigned16();
-    private Checksum crc32 = new PureJavaCrc32();
-
-    /**
-     * Get the final part of the input name
-     * @param split the input split
-     * @return the "part-r-00000" for the input
-     */
-    private String getFilename(FileSplit split) {
-      return split.getPath().getName();
-    }
-
-    public void map(Text key, Text value, Context context) 
-        throws IOException, InterruptedException {
-      if (lastKey == null) {
-        FileSplit fs = (FileSplit) context.getInputSplit();
-        filename = getFilename(fs);
-        context.write(new Text(filename + ":begin"), key);
-        lastKey = new Text();
-      } else {
-        if (key.compareTo(lastKey) < 0) {
-          context.write(ERROR, new Text("misorder in " + filename + 
-                                         " between " + textifyBytes(lastKey) + 
-                                         " and " + textifyBytes(key)));
-        }
-      }
-      // compute the crc of the key and value and add it to the sum
-      crc32.reset();
-      crc32.update(key.getBytes(), 0, key.getLength());
-      crc32.update(value.getBytes(), 0, value.getLength());
-      tmp.set(crc32.getValue());
-      checksum.add(tmp);
-      lastKey.set(key);
-    }
-    
-    public void cleanup(Context context) 
-        throws IOException, InterruptedException  {
-      if (lastKey != null) {
-        context.write(new Text(filename + ":end"), lastKey);
-        context.write(CHECKSUM, new Text(checksum.toString()));
-      }
-    }
-  }
-
-  /**
-   * Check the boundaries between the output files by making sure that the
-   * boundary keys are always increasing.
-   * Also passes any error reports along intact.
-   */
-  static class ValidateReducer extends Reducer<Text,Text,Text,Text> {
-    private boolean firstKey = true;
-    private Text lastKey = new Text();
-    private Text lastValue = new Text();
-    public void reduce(Text key, Iterable<Text> values,
-        Context context) throws IOException, InterruptedException  {
-      if (ERROR.equals(key)) {
-        for (Text val : values) {
-          context.write(key, val);
-        }
-      } else if (CHECKSUM.equals(key)) {
-        Unsigned16 tmp = new Unsigned16();
-        Unsigned16 sum = new Unsigned16();
-        for (Text val : values) {
-          tmp.set(val.toString());
-          sum.add(tmp);
-        }
-        context.write(CHECKSUM, new Text(sum.toString()));
-      } else {
-        Text value = values.iterator().next();
-        if (firstKey) {
-          firstKey = false;
-        } else {
-          if (value.compareTo(lastValue) < 0) {
-            context.write(ERROR, 
-                           new Text("bad key partitioning:\n  file " + 
-                                    lastKey + " key " + 
-                                    textifyBytes(lastValue) +
-                                    "\n  file " + key + " key " + 
-                                    textifyBytes(value)));
-          }
-        }
-        lastKey.set(key);
-        lastValue.set(value);
-      }
-    }
-    
-  }
-
-  private static void usage() throws IOException {
-    System.err.println("teravalidate <out-dir> <report-dir>");
-  }
-
-  public int run(String[] args) throws Exception {
-    Job job = Job.getInstance(getConf());
-    if (args.length != 2) {
-      usage();
-      return 1;
-    }
-    TeraInputFormat.setInputPaths(job, new Path(args[0]));
-    FileOutputFormat.setOutputPath(job, new Path(args[1]));
-    job.setJobName("TeraValidate");
-    job.setJarByClass(TeraValidate.class);
-    job.setMapperClass(ValidateMapper.class);
-    job.setReducerClass(ValidateReducer.class);
-    job.setOutputKeyClass(Text.class);
-    job.setOutputValueClass(Text.class);
-    // force a single reducer
-    job.setNumReduceTasks(1);
-    // force a single split 
-    FileInputFormat.setMinInputSplitSize(job, Long.MAX_VALUE);
-    job.setInputFormatClass(TeraInputFormat.class);
-    return job.waitForCompletion(true) ? 0 : 1;
-  }
-
-  /**
-   * @param args
-   */
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(new Configuration(), new TeraValidate(), args);
-    System.exit(res);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
deleted file mode 100644
index 1335bf4..0000000
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/terasort/Unsigned16.java
+++ /dev/null
@@ -1,297 +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.mapreduce.examples.terasort;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.apache.hadoop.io.Writable;
-
-/**
- * An unsigned 16 byte integer class that supports addition, multiplication,
- * and left shifts.
- */
-class Unsigned16 implements Writable {
-  private long hi8;
-  private long lo8;
-
-  public Unsigned16() {
-    hi8 = 0;
-    lo8 = 0;
-  }
-
-  public Unsigned16(long l) {
-    hi8 = 0;
-    lo8 = l;
-  }
-
-  public Unsigned16(Unsigned16 other) {
-    hi8 = other.hi8;
-    lo8 = other.lo8;
-  }
-  
-  @Override
-  public boolean equals(Object o) {
-    if (o instanceof Unsigned16) {
-      Unsigned16 other = (Unsigned16) o;
-      return other.hi8 == hi8 && other.lo8 == lo8;
-    }
-    return false;
-  }
-
-  @Override
-  public int hashCode() {
-    return (int) lo8;
-  }
-
-  /**
-   * Parse a hex string
-   * @param s the hex string
-   */
-  public Unsigned16(String s) throws NumberFormatException {
-    set(s);
-  }
-
-  /**
-   * Set the number from a hex string
-   * @param s the number in hexadecimal
-   * @throws NumberFormatException if the number is invalid
-   */
-  public void set(String s) throws NumberFormatException {
-    hi8 = 0;
-    lo8 = 0;
-    final long lastDigit = 0xfl << 60;
-    for (int i = 0; i < s.length(); ++i) {
-      int digit = getHexDigit(s.charAt(i));
-      if ((lastDigit & hi8) != 0) {
-        throw new NumberFormatException(s + " overflowed 16 bytes");
-      }
-      hi8 <<= 4;
-      hi8 |= (lo8 & lastDigit) >>> 60;
-      lo8 <<= 4;
-      lo8 |= digit;
-    }    
-  }
-
-  /**
-   * Set the number to a given long.
-   * @param l the new value, which is treated as an unsigned number
-   */
-  public void set(long l) {
-    lo8 = l;
-    hi8 = 0;
-  }
-
-  /**
-   * Map a hexadecimal character into a digit.
-   * @param ch the character
-   * @return the digit from 0 to 15
-   * @throws NumberFormatException
-   */
-  private static int getHexDigit(char ch) throws NumberFormatException {
-    if (ch >= '0' && ch <= '9') {
-      return ch - '0';
-    }
-    if (ch >= 'a' && ch <= 'f') {
-      return ch - 'a' + 10;
-    }
-    if (ch >= 'A' && ch <= 'F') {
-      return ch - 'A' + 10;
-    }
-    throw new NumberFormatException(ch + " is not a valid hex digit");
-  }
-
-  private static final Unsigned16 TEN = new Unsigned16(10);
-
-  public static Unsigned16 fromDecimal(String s) throws NumberFormatException {
-    Unsigned16 result = new Unsigned16();
-    Unsigned16 tmp = new Unsigned16();
-    for(int i=0; i < s.length(); i++) {
-      char ch = s.charAt(i);
-      if (ch < '0' || ch > '9') {
-        throw new NumberFormatException(ch + " not a valid decimal digit");
-      }
-      int digit = ch - '0';
-      result.multiply(TEN);
-      tmp.set(digit);
-      result.add(tmp);
-    }
-    return result;
-  }
-
-  /**
-   * Return the number as a hex string.
-   */
-  public String toString() {
-    if (hi8 == 0) {
-      return Long.toHexString(lo8);
-    } else {
-      StringBuilder result = new StringBuilder();
-      result.append(Long.toHexString(hi8));
-      String loString = Long.toHexString(lo8);
-      for(int i=loString.length(); i < 16; ++i) {
-        result.append('0');
-      }
-      result.append(loString);
-      return result.toString();
-    }
-  }
-
-  /**
-   * Get a given byte from the number.
-   * @param b the byte to get with 0 meaning the most significant byte
-   * @return the byte or 0 if b is outside of 0..15
-   */
-  public byte getByte(int b) {
-    if (b >= 0 && b < 16) {
-      if (b < 8) {
-        return (byte) (hi8 >> (56 - 8*b));
-      } else {
-        return (byte) (lo8 >> (120 - 8*b));
-      }
-    }
-    return 0;
-  }
-
-  /**
-   * Get the hexadecimal digit at the given position.
-   * @param p the digit position to get with 0 meaning the most significant
-   * @return the character or '0' if p is outside of 0..31
-   */
-  public char getHexDigit(int p) {
-    byte digit = getByte(p / 2);
-    if (p % 2 == 0) {
-      digit >>>= 4;
-    }
-    digit &= 0xf;
-    if (digit < 10) {
-      return (char) ('0' + digit);
-    } else {
-      return (char) ('A' + digit - 10);
-    }
-  }
-
-  /**
-   * Get the high 8 bytes as a long.
-   */
-  public long getHigh8() {
-    return hi8;
-  }
-  
-  /**
-   * Get the low 8 bytes as a long.
-   */
-  public long getLow8() {
-    return lo8;
-  }
-
-  /**
-   * Multiple the current number by a 16 byte unsigned integer. Overflow is not
-   * detected and the result is the low 16 bytes of the result. The numbers 
-   * are divided into 32 and 31 bit chunks so that the product of two chucks
-   * fits in the unsigned 63 bits of a long.
-   * @param b the other number
-   */
-  void multiply(Unsigned16 b) {
-    // divide the left into 4 32 bit chunks
-    long[] left = new long[4];
-    left[0] = lo8 & 0xffffffffl;
-    left[1] = lo8 >>> 32;
-    left[2] = hi8 & 0xffffffffl;
-    left[3] = hi8 >>> 32;
-    // divide the right into 5 31 bit chunks
-    long[] right = new long[5];
-    right[0] = b.lo8 & 0x7fffffffl;
-    right[1] = (b.lo8 >>> 31) & 0x7fffffffl;
-    right[2] = (b.lo8 >>> 62) + ((b.hi8 & 0x1fffffffl) << 2);
-    right[3] = (b.hi8 >>> 29) & 0x7fffffffl;
-    right[4] = (b.hi8 >>> 60);
-    // clear the cur value
-    set(0);
-    Unsigned16 tmp = new Unsigned16();
-    for(int l=0; l < 4; ++l) {
-      for (int r=0; r < 5; ++r) {
-        long prod = left[l] * right[r];
-        if (prod != 0) {
-          int off = l*32 + r*31;
-          tmp.set(prod);
-          tmp.shiftLeft(off);
-          add(tmp);
-        }
-      }
-    }
-  }
-
-  /**
-   * Add the given number into the current number.
-   * @param b the other number
-   */
-  public void add(Unsigned16 b) {
-    long sumHi;
-    long sumLo;
-    long  reshibit, hibit0, hibit1;
-
-    sumHi = hi8 + b.hi8;
-
-    hibit0 = (lo8 & 0x8000000000000000L);
-    hibit1 = (b.lo8 & 0x8000000000000000L);
-    sumLo = lo8 + b.lo8;
-    reshibit = (sumLo & 0x8000000000000000L);
-    if ((hibit0 & hibit1) != 0 | ((hibit0 ^ hibit1) != 0 && reshibit == 0))
-      sumHi++;  /* add carry bit */
-    hi8 = sumHi;
-    lo8 = sumLo;
-  }
-
-  /**
-   * Shift the number a given number of bit positions. The number is the low
-   * order bits of the result.
-   * @param bits the bit positions to shift by
-   */
-  public void shiftLeft(int bits) {
-    if (bits != 0) {
-      if (bits < 64) {
-        hi8 <<= bits;
-        hi8 |= (lo8 >>> (64 - bits));
-        lo8 <<= bits;
-      } else if (bits < 128) {
-        hi8 = lo8 << (bits - 64);
-        lo8 = 0;
-      } else {
-        hi8 = 0;
-        lo8 = 0;
-      }
-    }
-  }
-  
-  @Override
-  public void readFields(DataInput in) throws IOException {
-    hi8 = in.readLong();
-    lo8 = in.readLong();
-  }
-
-  @Override
-  public void write(DataOutput out) throws IOException {
-    out.writeLong(hi8);
-    out.writeLong(lo8);
-  }
-  
-  
-}

http://git-wip-us.apache.org/repos/asf/tez/blob/8e382b34/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java b/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java
deleted file mode 100644
index 97b6538..0000000
--- a/tez-tests/src/test/java/org/apache/tez/mapreduce/examples/terasort/TestTeraSort.java
+++ /dev/null
@@ -1,94 +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.mapreduce.examples.terasort;
-
-import java.io.File;
-import java.io.IOException;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.mapred.HadoopTestCase;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.tez.mapreduce.examples.terasort.TeraGen;
-import org.apache.tez.mapreduce.examples.terasort.TeraSort;
-import org.apache.tez.mapreduce.examples.terasort.TeraValidate;
-
-import org.junit.Ignore;
-
-@Ignore
-public class TestTeraSort extends HadoopTestCase {
-  
-  public TestTeraSort()
-      throws IOException {
-    super(CLUSTER_MR, DFS_FS, 1, 1);
-  }
-
-  protected void tearDown() throws Exception {
-    getFileSystem().delete(new Path(TEST_DIR), true);
-    super.tearDown();
-  }
-  
-  // Input/Output paths for sort
-  private static final String TEST_DIR = 
-    new File(System.getProperty("test.build.data", "/tmp"), "org/apache/tez/mapreduce/examples/terasort")
-    .getAbsolutePath();
-  private static final Path SORT_INPUT_PATH = new Path(TEST_DIR, "sortin");
-  private static final Path SORT_OUTPUT_PATH = new Path(TEST_DIR, "sortout");
-  private static final Path TERA_OUTPUT_PATH = new Path(TEST_DIR, "validate");
-  private static final String NUM_ROWS = "100"; 
-
-  private void runTeraGen(Configuration conf, Path sortInput) 
-      throws Exception {
-    String[] genArgs = {NUM_ROWS, sortInput.toString()};
-    
-    // Run TeraGen
-    assertEquals(ToolRunner.run(conf, new TeraGen(), genArgs), 0);
-  }
-  
-  private void runTeraSort(Configuration conf,
-      Path sortInput, Path sortOutput) throws Exception {
-
-    // Setup command-line arguments to 'sort'
-    String[] sortArgs = {sortInput.toString(), sortOutput.toString()};
-    
-    // Run Sort
-    assertEquals(ToolRunner.run(conf, new TeraSort(), sortArgs), 0);
-  }
-  
-  private void runTeraValidator(Configuration job, 
-                                       Path sortOutput, Path valOutput) 
-  throws Exception {
-    String[] svArgs = {sortOutput.toString(), valOutput.toString()};
-
-    // Run Tera-Validator
-    assertEquals(ToolRunner.run(job, new TeraValidate(), svArgs), 0);
-  }
-  
-  public void testTeraSort() throws Exception {
-    // Run TeraGen to generate input for 'terasort'
-    runTeraGen(createJobConf(), SORT_INPUT_PATH);
-
-    // Run terasort
-    runTeraSort(createJobConf(), SORT_INPUT_PATH, SORT_OUTPUT_PATH);
-
-    // Run tera-validator to check if sort worked correctly
-    runTeraValidator(createJobConf(), SORT_OUTPUT_PATH,
-      TERA_OUTPUT_PATH);
-  }
-
-}


[25/25] git commit: TEZ-1157. Optimize broadcast shuffle to download data only once per host. (gopalv)

Posted by je...@apache.org.
TEZ-1157. Optimize broadcast shuffle to download data only once per host. (gopalv)


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

Branch: refs/heads/TEZ-8
Commit: 625450cf11454fa9697a902ba70367de00cdc170
Parents: e328055
Author: Gopal V <go...@apache.org>
Authored: Wed Sep 17 20:53:11 2014 -0700
Committer: Gopal V <go...@apache.org>
Committed: Wed Sep 17 20:53:11 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   4 +-
 .../library/api/TezRuntimeConfiguration.java    |  12 +
 .../library/common/InputAttemptIdentifier.java  |  26 +-
 .../shuffle/common/DiskFetchedInput.java        |   7 +
 .../library/shuffle/common/FetchedInput.java    |   3 +-
 .../shuffle/common/FetchedInputAllocator.java   |   3 +
 .../runtime/library/shuffle/common/Fetcher.java | 318 +++++++++++++++++--
 .../impl/ShuffleInputEventHandlerImpl.java      |  18 +-
 .../shuffle/common/impl/ShuffleManager.java     |  42 ++-
 .../impl/SimpleFetchedInputAllocator.java       |  17 +
 10 files changed, 412 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index bd5569c..5e2c2cd 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -19,8 +19,8 @@ ALL CHANGES:
   TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
   TEZ-1524. Resolve user group information only if ACLs are enabled.
   TEZ-1581. GroupByOrderByMRRTest no longer functional.
-  TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
-  of DAG submission
+  TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless of DAG submission.
+  TEZ-1157. Optimize broadcast shuffle to download data only once per host. 
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
index 077ce8e..cb61109 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/api/TezRuntimeConfiguration.java
@@ -280,6 +280,17 @@ public class TezRuntimeConfiguration {
    */
   public static final boolean TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH_DEFAULT = false;
 
+  /**
+   * Share data fetched between tasks running on the same host if applicable
+   */
+  public static final String TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH = TEZ_RUNTIME_PREFIX
+      + "optimize.shared.fetch";
+
+  /**
+   * shared mode bypassing the http fetch is not enabled by default till we have unit tests in.
+   */
+  public static final boolean TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH_DEFAULT = false;
+
   // TODO TEZ-1233 - allow this property to be set per vertex
   // TODO TEZ-1231 - move these properties out since they are not relevant for Inputs / Outputs
 
@@ -333,6 +344,7 @@ public class TezRuntimeConfiguration {
     tezRuntimeKeys.add(TEZ_RUNTIME_TRANSFER_DATA_VIA_EVENTS_MAX_SIZE);
     tezRuntimeKeys.add(TEZ_RUNTIME_RECORDS_BEFORE_PROGRESS);
     tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH);
+    tezRuntimeKeys.add(TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH);
 
     defaultConf.addResource("core-default.xml");
     defaultConf.addResource("core-site.xml");

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
index 7c8a23b..9987d26 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/common/InputAttemptIdentifier.java
@@ -29,28 +29,38 @@ public class InputAttemptIdentifier {
 
   private final InputIdentifier inputIdentifier;
   private final int attemptNumber;
-  private String pathComponent;
-  
+  private final String pathComponent;
+  private final boolean shared;
+
   public static final String PATH_PREFIX = "attempt";
-  
+
   public InputAttemptIdentifier(int inputIndex, int attemptNumber) {
     this(new InputIdentifier(inputIndex), attemptNumber, null);
   }
-  
+
   public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent) {
+    this(inputIdentifier, attemptNumber, pathComponent, false);
+  }
+
+  public InputAttemptIdentifier(InputIdentifier inputIdentifier, int attemptNumber, String pathComponent, boolean shared) {
     this.inputIdentifier = inputIdentifier;
     this.attemptNumber = attemptNumber;
     this.pathComponent = pathComponent;
+    this.shared = shared;
     if (pathComponent != null && !pathComponent.startsWith(PATH_PREFIX)) {
       throw new TezUncheckedException(
           "Path component must start with: " + PATH_PREFIX + " " + this);
     }
   }
-  
+
   public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent) {
     this(new InputIdentifier(taskIndex), attemptNumber, pathComponent);
   }
 
+  public InputAttemptIdentifier(int taskIndex, int attemptNumber, String pathComponent, boolean shared) {
+    this(new InputIdentifier(taskIndex), attemptNumber, pathComponent, shared);
+  }
+
   public InputIdentifier getInputIdentifier() {
     return this.inputIdentifier;
   }
@@ -63,7 +73,11 @@ public class InputAttemptIdentifier {
     return pathComponent;
   }
 
-  // PathComponent does not need to be part of the hashCode and equals computation.
+  public boolean isShared() {
+    return this.shared;
+  }
+
+  // PathComponent & shared does not need to be part of the hashCode and equals computation.
   @Override
   public int hashCode() {
     final int prime = 31;

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
index 1d26c6e..b0b911b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/DiskFetchedInput.java
@@ -65,6 +65,13 @@ public class DiskFetchedInput extends FetchedInput {
   public InputStream getInputStream() throws IOException {
     return localFS.open(outputPath);
   }
+
+  public final Path getInputPath() {
+    if (state == State.COMMITTED) {
+      return this.outputPath;
+    }
+    return this.tmpOutputPath;
+  }
   
   @Override
   public void commit() throws IOException {

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
index fde19b7..0a83dc9 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInput.java
@@ -110,7 +110,8 @@ public abstract class FetchedInput {
 
   /**
    * Return an input stream to be used to read the previously fetched data.
-   * Users are expected to close the InputStream when they're done
+   * All calls to getInputStream() produce new reset streams for reading.
+   * Users are expected to close the InputStream when they're done.
    */
   public abstract InputStream getInputStream() throws IOException;
 

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
index 1707ab7..288df6d 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/FetchedInputAllocator.java
@@ -21,11 +21,14 @@ package org.apache.tez.runtime.library.shuffle.common;
 import java.io.IOException;
 
 import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 
 public interface FetchedInputAllocator {
 
   public FetchedInput allocate(long actualSize, long compresedSize,
       InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
   
+  public FetchedInput allocateType(Type type, long actualSize, long compresedSize,
+      InputAttemptIdentifier inputAttemptIdentifier) throws IOException;
   
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
index 9cb8617..e25124b 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/Fetcher.java
@@ -19,10 +19,17 @@
 package org.apache.tez.runtime.library.shuffle.common;
 
 import java.io.DataInputStream;
+import java.io.File;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.RandomAccessFile;
 import java.net.URL;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
 import java.util.Arrays;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.Iterator;
 import java.util.LinkedHashSet;
 import java.util.List;
@@ -34,12 +41,19 @@ import java.util.concurrent.atomic.AtomicInteger;
 import javax.crypto.SecretKey;
 
 import com.google.common.annotations.VisibleForTesting;
+
+import org.apache.commons.io.IOUtils;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.LocalDirAllocator;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.hadoop.util.DiskChecker.DiskErrorException;
 import org.apache.hadoop.yarn.api.ApplicationConstants;
 import org.apache.hadoop.yarn.api.records.ApplicationId;
 import org.apache.tez.common.TezRuntimeFrameworkConfigs;
@@ -54,6 +68,7 @@ import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 import org.apache.tez.runtime.library.shuffle.common.HttpConnection.HttpConnectionParams;
 
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 
 /**
  * Responsible for fetching inputs served by the ShuffleHandler for a single
@@ -101,10 +116,22 @@ public class Fetcher implements Callable<FetchResult> {
   private HttpConnectionParams httpConnectionParams;
 
   private final boolean localDiskFetchEnabled;
+  private final boolean sharedFetchEnabled;
+
+  private final LocalDirAllocator localDirAllocator;
+  private final Path lockPath;
+  private final RawLocalFileSystem localFs;
+
+  private final boolean isDebugEnabled = LOG.isDebugEnabled();
 
   private Fetcher(FetcherCallback fetcherCallback, HttpConnectionParams params,
-      FetchedInputAllocator inputManager, ApplicationId appId, SecretKey shuffleSecret,
-      String srcNameTrimmed, Configuration conf, boolean localDiskFetchEnabled) {
+      FetchedInputAllocator inputManager, ApplicationId appId,
+      SecretKey shuffleSecret, String srcNameTrimmed, Configuration conf,
+      RawLocalFileSystem localFs,
+      LocalDirAllocator localDirAllocator,
+      Path lockPath,
+      boolean localDiskFetchEnabled,
+      boolean sharedFetchEnabled) {
     this.fetcherCallback = fetcherCallback;
     this.inputManager = inputManager;
     this.shuffleSecret = shuffleSecret;
@@ -114,19 +141,42 @@ public class Fetcher implements Callable<FetchResult> {
     this.conf = conf;
 
     this.localDiskFetchEnabled = localDiskFetchEnabled;
+    this.sharedFetchEnabled = sharedFetchEnabled;
 
     this.fetcherIdentifier = fetcherIdGen.getAndIncrement();
-    this.logIdentifier = "fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier;
+    this.logIdentifier = " fetcher [" + srcNameTrimmed +"] " + fetcherIdentifier;
+
+    this.localFs = localFs;
+    this.localDirAllocator = localDirAllocator;
+    this.lockPath = lockPath;
+
+    try {
+      if (this.sharedFetchEnabled) {
+        this.localFs.mkdirs(this.lockPath);
+      }
+    } catch (Exception e) {
+      LOG.warn("Error initializing local dirs for shared transfer " + e);
+    }
   }
 
   @Override
   public FetchResult call() throws Exception {
+    boolean multiplex = (this.sharedFetchEnabled && this.localDiskFetchEnabled);
+
     if (srcAttempts.size() == 0) {
       return new FetchResult(host, port, partition, srcAttempts);
     }
 
     for (InputAttemptIdentifier in : srcAttempts) {
       pathToAttemptMap.put(in.getPathComponent(), in);
+      // do only if all of them are shared fetches
+      multiplex &= in.isShared();
+    }
+
+    if (multiplex) {
+      Preconditions.checkArgument(partition == 0,
+          "Shared fetches cannot be done for partitioned input"
+              + "- partition is non-zero (%d)", partition);
     }
 
     remaining = new LinkedHashSet<InputAttemptIdentifier>(srcAttempts);
@@ -136,7 +186,9 @@ public class Fetcher implements Callable<FetchResult> {
     if (localDiskFetchEnabled &&
         host.equals(System.getenv(ApplicationConstants.Environment.NM_HOST.toString()))) {
       hostFetchResult = setupLocalDiskFetch();
-    } else {
+    } else if (multiplex) {
+      hostFetchResult = doSharedFetch();
+    } else{
       hostFetchResult = doHttpFetch();
     }
 
@@ -151,15 +203,197 @@ public class Fetcher implements Callable<FetchResult> {
 
     // Sanity check
     if (hostFetchResult.failedInputs == null && !remaining.isEmpty()) {
-      throw new IOException("server didn't return all expected map outputs: "
-          + remaining.size() + " left.");
+      if (!multiplex) {
+        throw new IOException("server didn't return all expected map outputs: "
+            + remaining.size() + " left.");
+      } else {
+        LOG.info("Shared fetch failed to return " + remaining.size() + " inputs on this try");
+      }
     }
 
     return hostFetchResult.fetchResult;
   }
 
+  private final class CachingCallBack {
+    // this is a closure object wrapping this in an inner class
+    public void cache(String host,
+        InputAttemptIdentifier srcAttemptId, FetchedInput fetchedInput,
+        long compressedLength, long decompressedLength) {
+      try {
+        // this breaks badly on partitioned input - please use responsibly
+        Preconditions.checkArgument(partition == 0, "Partition == 0");
+        final String tmpSuffix = "." + System.currentTimeMillis() + ".tmp";
+        final String finalOutput = getMapOutputFile(srcAttemptId.getPathComponent());
+        final Path outputPath = localDirAllocator.getLocalPathForWrite(finalOutput, compressedLength, conf);
+        final TezSpillRecord spillRec = new TezSpillRecord(1);
+        final TezIndexRecord indexRec;
+        Path tmpIndex = outputPath.suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING+tmpSuffix);
+
+        if (localFs.exists(tmpIndex)) {
+          LOG.warn("Found duplicate instance of input index file " + tmpIndex);
+          return;
+        }
+
+        Path tmpPath = null;
+
+        switch (fetchedInput.getType()) {
+        case DISK: {
+          DiskFetchedInput input = (DiskFetchedInput) fetchedInput;
+          indexRec = new TezIndexRecord(0, decompressedLength, compressedLength);
+          localFs.mkdirs(outputPath.getParent());
+          // avoid pit-falls of speculation
+          tmpPath = outputPath.suffix(tmpSuffix);
+          // JDK7 - TODO: use Files implementation to speed up this process
+          localFs.copyFromLocalFile(input.getInputPath(), tmpPath);
+          // rename is atomic
+          boolean renamed = localFs.rename(tmpPath, outputPath);
+          if(!renamed) {
+            LOG.warn("Could not rename to cached file name " + outputPath);
+            localFs.delete(tmpPath, false);
+            return;
+          }
+        }
+        break;
+        default:
+          LOG.warn("Incorrect use of CachingCallback for " + srcAttemptId);
+          return;
+        }
+
+        spillRec.putIndex(indexRec, 0);
+        spillRec.writeToFile(tmpIndex, conf);
+        // everything went well so far - rename it
+        boolean renamed = localFs.rename(tmpIndex, outputPath
+            .suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING));
+        if (!renamed) {
+          localFs.delete(tmpIndex, false);
+          if (outputPath != null) {
+            // invariant: outputPath was renamed from tmpPath
+            localFs.delete(outputPath, false);
+          }
+          LOG.warn("Could not rename the index file to "
+              + outputPath
+                  .suffix(Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING));
+          return;
+        }
+      } catch (IOException ioe) {
+        // do mostly nothing
+        LOG.warn("Cache threw an error " + ioe);
+      }
+    }
+  }
+
+  private int findInputs() throws IOException {
+    int k = 0;
+    for (InputAttemptIdentifier src : srcAttempts) {
+      try {
+        if (getShuffleInputFileName(src.getPathComponent(),
+            Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING) != null) {
+          k++;
+        }
+      } catch (DiskErrorException de) {
+        // missing file, ignore
+      }
+    }
+    return k;
+  }
+
+  private FileLock getLock() throws OverlappingFileLockException, InterruptedException, IOException {
+    File lockFile = localFs.pathToFile(new Path(lockPath, host + ".lock"));
+
+    final boolean created = lockFile.createNewFile();
+
+    if (created == false && !lockFile.exists()) {
+      // bail-out cleanly
+      return null;
+    }
+
+    // invariant - file created (winner writes to this file)
+    // caveat: closing lockChannel does close the file (do not double close)
+    // JDK7 - TODO: use AsynchronousFileChannel instead of RandomAccessFile
+    FileChannel lockChannel = new RandomAccessFile(lockFile, "rws")
+        .getChannel();
+    FileLock xlock = null;
+
+    xlock = lockChannel.tryLock(0, Long.MAX_VALUE, false);
+    if (xlock != null) {
+      return xlock;
+    }
+    lockChannel.close();
+    return null;
+  }
+
+  private void releaseLock(FileLock lock) throws IOException {
+    if (lock != null && lock.isValid()) {
+      FileChannel lockChannel = lock.channel();
+      lock.release();
+      lockChannel.close();
+    }
+  }
+
+  protected HostFetchResult doSharedFetch() throws IOException {
+    int inputs = findInputs();
+
+    if (inputs == srcAttempts.size()) {
+      if (isDebugEnabled) {
+        LOG.debug("Using the copies found locally");
+      }
+      return doLocalDiskFetch(true);
+    }
+
+    if (inputs > 0) {
+      if (isDebugEnabled) {
+        LOG.debug("Found " + input
+            + " local fetches right now, using them first");
+      }
+      return doLocalDiskFetch(false);
+    }
+
+    FileLock lock = null;
+    try {
+      lock = getLock();
+      if (lock == null) {
+        // re-queue until we get a lock
+        LOG.info("Requeuing " + host + ":" + port
+            + " downloads because we didn't get a lock");
+        return new HostFetchResult(new FetchResult(host, port, partition,
+            remaining), null, false);
+      } else {
+        if (findInputs() == srcAttempts.size()) {
+          // double checked after lock
+          releaseLock(lock);
+          lock = null;
+          return doLocalDiskFetch(true);
+        }
+        // cache data if possible
+        return doHttpFetch(new CachingCallBack());
+      }
+    } catch (OverlappingFileLockException jvmCrossLock) {
+      // fall back to HTTP fetch below
+      LOG.warn("Double locking detected for " + host);
+    } catch (InterruptedException sleepInterrupted) {
+      // fall back to HTTP fetch below
+      LOG.warn("Lock was interrupted for " + host);
+    } finally {
+      releaseLock(lock);
+    }
+
+    if (isShutDown.get()) {
+      // if any exception was due to shut-down don't bother firing any more
+      // requests
+      return new HostFetchResult(new FetchResult(host, port, partition,
+          remaining), null, false);
+    }
+    // no more caching
+    return doHttpFetch();
+  }
+
   @VisibleForTesting
   protected HostFetchResult doHttpFetch() {
+    return doHttpFetch(null);
+  }
+
+  @VisibleForTesting
+  protected HostFetchResult doHttpFetch(CachingCallBack callback) {
     try {
       StringBuilder baseURI = ShuffleUtils.constructBaseURIForShuffleHandler(host,
           port, partition, appId.toString(), httpConnectionParams.isSSLShuffleEnabled());
@@ -227,7 +461,7 @@ public class Fetcher implements Callable<FetchResult> {
     // yet_to_be_fetched list and marking the failed tasks.
     InputAttemptIdentifier[] failedInputs = null;
     while (!remaining.isEmpty() && failedInputs == null) {
-      failedInputs = fetchInputs(input);
+      failedInputs = fetchInputs(input, callback);
     }
 
     return new HostFetchResult(new FetchResult(host, port, partition, remaining), failedInputs,
@@ -236,6 +470,11 @@ public class Fetcher implements Callable<FetchResult> {
 
   @VisibleForTesting
   protected HostFetchResult setupLocalDiskFetch() {
+    return doLocalDiskFetch(true);
+  }
+
+  @VisibleForTesting
+  private HostFetchResult doLocalDiskFetch(boolean failMissing) {
 
     Iterator<InputAttemptIdentifier> iterator = remaining.iterator();
     while (iterator.hasNext()) {
@@ -246,6 +485,7 @@ public class Fetcher implements Callable<FetchResult> {
       FetchedInput fetchedInput = null;
       try {
         TezIndexRecord idxRecord;
+        // for missing files, this will throw an exception
         idxRecord = getTezIndexRecord(srcAttemptId);
 
         fetchedInput = new LocalDiskFetchedInput(idxRecord.getStartOffset(),
@@ -283,8 +523,10 @@ public class Fetcher implements Callable<FetchResult> {
     }
 
     InputAttemptIdentifier[] failedFetches = null;
-    if (remaining.size() > 0) {
+    if (failMissing && remaining.size() > 0) {
       failedFetches = remaining.toArray(new InputAttemptIdentifier[remaining.size()]);
+    } else {
+      // nothing needs to be done to requeue remaining entries
     }
     return new HostFetchResult(new FetchResult(host, port, partition, remaining),
         failedFetches, false);
@@ -296,19 +538,24 @@ public class Fetcher implements Callable<FetchResult> {
     TezIndexRecord idxRecord;
     Path indexFile = getShuffleInputFileName(srcAttemptId.getPathComponent(),
         Constants.TEZ_RUNTIME_TASK_OUTPUT_INDEX_SUFFIX_STRING);
+
     TezSpillRecord spillRecord = new TezSpillRecord(indexFile, conf);
     idxRecord = spillRecord.getIndex(partition);
     return idxRecord;
   }
 
+  private static final String getMapOutputFile(String pathComponent) {
+    return Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR
+        + pathComponent + Path.SEPARATOR
+        + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING;
+  }
+
   @VisibleForTesting
-  protected Path getShuffleInputFileName(String pathComponent, String suffix) throws IOException {
-    LocalDirAllocator localDirAllocator = new LocalDirAllocator(TezRuntimeFrameworkConfigs.LOCAL_DIRS);
+  protected Path getShuffleInputFileName(String pathComponent, String suffix)
+      throws IOException {
     suffix = suffix != null ? suffix : "";
 
-    String pathFromLocalDir = Constants.TEZ_RUNTIME_TASK_OUTPUT_DIR + Path.SEPARATOR + pathComponent +
-        Path.SEPARATOR + Constants.TEZ_RUNTIME_TASK_OUTPUT_FILENAME_STRING + suffix;
-
+    String pathFromLocalDir = getMapOutputFile(pathComponent) + suffix;
     return localDirAllocator.getLocalPathToRead(pathFromLocalDir, conf);
   }
 
@@ -350,7 +597,7 @@ public class Fetcher implements Callable<FetchResult> {
     }
   }
 
-  private InputAttemptIdentifier[] fetchInputs(DataInputStream input) {
+  private InputAttemptIdentifier[] fetchInputs(DataInputStream input, CachingCallBack callback) {
     FetchedInput fetchedInput = null;
     InputAttemptIdentifier srcAttemptId = null;
     long decompressedLength = -1;
@@ -392,12 +639,16 @@ public class Fetcher implements Callable<FetchResult> {
         LOG.debug("header: " + srcAttemptId + ", len: " + compressedLength
             + ", decomp len: " + decompressedLength);
       }
-
-      // Get the location for the map output - either in-memory or on-disk
       
       // TODO TEZ-957. handle IOException here when Broadcast has better error checking
-      fetchedInput = inputManager.allocate(decompressedLength, compressedLength, srcAttemptId);
-
+      if (srcAttemptId.isShared() && callback != null) {
+        // force disk if input is being shared
+        fetchedInput = inputManager.allocateType(Type.DISK, decompressedLength,
+            compressedLength, srcAttemptId);
+      } else {
+        fetchedInput = inputManager.allocate(decompressedLength,
+            compressedLength, srcAttemptId);
+      }
       // TODO NEWTEZ No concept of WAIT at the moment.
       // // Check if we can shuffle *now* ...
       // if (fetchedInput.getType() == FetchedInput.WAIT) {
@@ -427,6 +678,14 @@ public class Fetcher implements Callable<FetchResult> {
             fetchedInput);
       }
 
+      // offer the fetched input for caching
+      if (srcAttemptId.isShared() && callback != null) {
+        // this has to be before the fetchSucceeded, because that goes across
+        // threads into the reader thread and can potentially shutdown this thread
+        // while it is still caching.
+        callback.cache(host, srcAttemptId, fetchedInput, compressedLength, decompressedLength);
+      }
+
       // Inform the shuffle scheduler
       long endTime = System.currentTimeMillis();
       fetcherCallback.fetchSucceeded(host, srcAttemptId, fetchedInput,
@@ -434,6 +693,7 @@ public class Fetcher implements Callable<FetchResult> {
 
       // Note successful shuffle
       remaining.remove(srcAttemptId);
+
       // metrics.successFetch();
       return null;
     } catch (IOException ioe) {
@@ -521,12 +781,24 @@ public class Fetcher implements Callable<FetchResult> {
     private Fetcher fetcher;
     private boolean workAssigned = false;
 
-    public FetcherBuilder(FetcherCallback fetcherCallback, HttpConnectionParams params,
-                          FetchedInputAllocator inputManager, ApplicationId appId,
-                          SecretKey shuffleSecret, String srcNameTrimmed, Configuration conf,
-                          boolean localDiskFetchEnabled) {
+    public FetcherBuilder(FetcherCallback fetcherCallback,
+        HttpConnectionParams params, FetchedInputAllocator inputManager,
+        ApplicationId appId, SecretKey shuffleSecret, String srcNameTrimmed,
+        Configuration conf, boolean localDiskFetchEnabled) {
+      this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
+          shuffleSecret, srcNameTrimmed, conf, null, null, null, localDiskFetchEnabled,
+          false);
+    }
+
+    public FetcherBuilder(FetcherCallback fetcherCallback,
+        HttpConnectionParams params, FetchedInputAllocator inputManager,
+        ApplicationId appId, SecretKey shuffleSecret, String srcNameTrimmed,
+        Configuration conf, RawLocalFileSystem localFs,
+        LocalDirAllocator localDirAllocator, Path lockPath,
+        boolean localDiskFetchEnabled, boolean sharedFetchEnabled) {
       this.fetcher = new Fetcher(fetcherCallback, params, inputManager, appId,
-          shuffleSecret, srcNameTrimmed, conf, localDiskFetchEnabled);
+          shuffleSecret, srcNameTrimmed, conf, localFs, localDirAllocator,
+          lockPath, localDiskFetchEnabled, sharedFetchEnabled);
     }
 
     public FetcherBuilder setHttpConnectionParameters(HttpConnectionParams httpParams) {

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
index 9d621e8..2ac45d4 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleInputEventHandlerImpl.java
@@ -24,6 +24,7 @@ import java.util.BitSet;
 import java.util.List;
 
 import com.google.protobuf.ByteString;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.io.compress.CompressionCodec;
@@ -53,6 +54,7 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
   private final CompressionCodec codec;
   private final boolean ifileReadAhead;
   private final int ifileReadAheadLength;
+  private final boolean useSharedInputs;
 
   public ShuffleInputEventHandlerImpl(InputContext inputContext,
                                       ShuffleManager shuffleManager,
@@ -63,6 +65,9 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
     this.codec = codec;
     this.ifileReadAhead = ifileReadAhead;
     this.ifileReadAheadLength = ifileReadAheadLength;
+    // this currently relies on a user to enable the flag
+    // expand on idea based on vertex parallelism and num inputs
+    this.useSharedInputs = (inputContext.getTaskAttemptNumber() == 0);
   }
 
   @Override
@@ -109,9 +114,12 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
         shuffleManager.addCompletedInputWithNoData(srcAttemptIdentifier);
         return;
       }
-    } 
-    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(dme.getTargetIndex(),
-        dme.getVersion(), shufflePayload.getPathComponent());
+    }
+
+    InputAttemptIdentifier srcAttemptIdentifier = new InputAttemptIdentifier(
+        dme.getTargetIndex(), dme.getVersion(),
+        shufflePayload.getPathComponent(), (useSharedInputs && srcIndex == 0));
+
     if (shufflePayload.hasData()) {
       DataProto dataProto = shufflePayload.getData();
       FetchedInput fetchedInput = inputAllocator.allocate(dataProto.getRawLength(),
@@ -119,8 +127,8 @@ public class ShuffleInputEventHandlerImpl implements ShuffleEventHandler {
       moveDataToFetchedInput(dataProto, fetchedInput, hostIdentifier);
       shuffleManager.addCompletedInputWithData(srcAttemptIdentifier, fetchedInput);
     } else {
-      shuffleManager.addKnownInput(shufflePayload.getHost(), shufflePayload.getPort(),
-              srcAttemptIdentifier, srcIndex);
+      shuffleManager.addKnownInput(shufflePayload.getHost(),
+          shufflePayload.getPort(), srcAttemptIdentifier, srcIndex);
     }
 
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
index 4e1a06c..8aa6582 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/ShuffleManager.java
@@ -22,6 +22,7 @@ import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
 import java.text.DecimalFormat;
+import java.util.Arrays;
 import java.util.Collections;
 import java.util.Iterator;
 import java.util.List;
@@ -43,7 +44,12 @@ import javax.crypto.SecretKey;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.LocalDirAllocator;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.RawLocalFileSystem;
 import org.apache.hadoop.io.compress.CompressionCodec;
+import org.apache.tez.common.TezRuntimeFrameworkConfigs;
 import org.apache.tez.common.TezUtilsInternal;
 import org.apache.tez.common.counters.TaskCounter;
 import org.apache.tez.common.counters.TezCounter;
@@ -68,7 +74,9 @@ import org.apache.tez.runtime.library.shuffle.common.ShuffleUtils;
 import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 import org.apache.tez.runtime.library.shuffle.common.Fetcher.FetcherBuilder;
 
+import com.google.common.base.Objects;
 import com.google.common.base.Preconditions;
+import com.google.common.collect.Iterables;
 import com.google.common.collect.Lists;
 import com.google.common.util.concurrent.FutureCallback;
 import com.google.common.util.concurrent.Futures;
@@ -120,6 +128,7 @@ public class ShuffleManager implements FetcherCallback {
   private final SecretKey shuffleSecret;
   private final CompressionCodec codec;
   private final boolean localDiskFetchEnabled;
+  private final boolean sharedFetchEnabled;
   
   private final int ifileBufferSize;
   private final boolean ifileReadAhead;
@@ -140,6 +149,11 @@ public class ShuffleManager implements FetcherCallback {
   private volatile Throwable shuffleError;
   private final HttpConnectionParams httpConnectionParams;
   
+
+  private final LocalDirAllocator localDirAllocator;
+  private final RawLocalFileSystem localFs;
+  private final Path[] localDisks;
+
   // TODO More counters - FetchErrors, speed?
   
   public ShuffleManager(InputContext inputContext, Configuration conf, int numInputs,
@@ -163,6 +177,8 @@ public class ShuffleManager implements FetcherCallback {
     this.inputManager = inputAllocator;
     this.localDiskFetchEnabled = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH,
         TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_LOCAL_FETCH_DEFAULT);
+    this.sharedFetchEnabled = conf.getBoolean(TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH,
+        TezRuntimeConfiguration.TEZ_RUNTIME_OPTIMIZE_SHARED_FETCH_DEFAULT);
     
     this.srcNameTrimmed = TezUtilsInternal.cleanVertexName(inputContext.getSourceVertexName());
   
@@ -199,10 +215,23 @@ public class ShuffleManager implements FetcherCallback {
             .getServiceConsumerMetaData(TezConstants.TEZ_SHUFFLE_HANDLER_SERVICE_ID));
     httpConnectionParams =
         ShuffleUtils.constructHttpShuffleConnectionParams(conf);
+
+    this.localFs = (RawLocalFileSystem) FileSystem.getLocal(conf).getRaw();
+
+    this.localDirAllocator = new LocalDirAllocator(
+        TezRuntimeFrameworkConfigs.LOCAL_DIRS);
+
+    this.localDisks = Iterables.toArray(
+        localDirAllocator.getAllLocalPathsToRead(".", conf), Path.class);
+
+    Arrays.sort(this.localDisks);
+
     LOG.info(this.getClass().getSimpleName() + " : numInputs=" + numInputs + ", compressionCodec="
         + (codec == null ? "NoCompressionCodec" : codec.getClass().getName()) + ", numFetchers="
         + numFetchers + ", ifileBufferSize=" + ifileBufferSize + ", ifileReadAheadEnabled="
         + ifileReadAhead + ", ifileReadAheadLength=" + ifileReadAheadLength +", "
+        + "localDiskFetchEnabled=" + localDiskFetchEnabled + ", "
+        + "sharedFetchEnabled=" + sharedFetchEnabled + ", "
         + httpConnectionParams.toString());
   }
 
@@ -301,9 +330,20 @@ public class ShuffleManager implements FetcherCallback {
   }
   
   private Fetcher constructFetcherForHost(InputHost inputHost, Configuration conf) {
+
+    Path lockDisk = null;
+
+    if (sharedFetchEnabled) {
+      // pick a single lock disk from the edge name's hashcode + host hashcode
+      final int h = Math.abs(Objects.hashCode(this.srcNameTrimmed, inputHost.getHost()));
+      lockDisk = new Path(this.localDisks[h % this.localDisks.length], "locks");
+    }
+
     FetcherBuilder fetcherBuilder = new FetcherBuilder(ShuffleManager.this,
       httpConnectionParams, inputManager, inputContext.getApplicationId(),
-      shuffleSecret, srcNameTrimmed, conf, localDiskFetchEnabled);
+        shuffleSecret, srcNameTrimmed, conf, localFs, localDirAllocator,
+        lockDisk, localDiskFetchEnabled, sharedFetchEnabled);
+
     if (codec != null) {
       fetcherBuilder.setCompressionParameters(codec);
     }

http://git-wip-us.apache.org/repos/asf/tez/blob/625450cf/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
index d365aa4..20ee665 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/impl/SimpleFetchedInputAllocator.java
@@ -33,6 +33,7 @@ import org.apache.tez.runtime.library.common.InputAttemptIdentifier;
 import org.apache.tez.runtime.library.common.task.local.output.TezTaskOutputFiles;
 import org.apache.tez.runtime.library.shuffle.common.DiskFetchedInput;
 import org.apache.tez.runtime.library.shuffle.common.FetchedInput;
+import org.apache.tez.runtime.library.shuffle.common.FetchedInput.Type;
 import org.apache.tez.runtime.library.shuffle.common.FetchedInputAllocator;
 import org.apache.tez.runtime.library.shuffle.common.FetchedInputCallback;
 import org.apache.tez.runtime.library.shuffle.common.MemoryFetchedInput;
@@ -140,10 +141,26 @@ public class SimpleFetchedInputAllocator implements FetchedInputAllocator,
   }
 
   @Override
+  public synchronized FetchedInput allocateType(Type type, long actualSize,
+      long compressedSize, InputAttemptIdentifier inputAttemptIdentifier)
+      throws IOException {
+
+    switch (type) {
+    case DISK:
+      return new DiskFetchedInput(actualSize, compressedSize,
+          inputAttemptIdentifier, this, conf, localDirAllocator,
+          fileNameAllocator);
+    default:
+      return allocate(actualSize, compressedSize, inputAttemptIdentifier);
+    }
+  }
+
+  @Override
   public synchronized void fetchComplete(FetchedInput fetchedInput) {
     switch (fetchedInput.getType()) {
     // Not tracking anything here.
     case DISK:
+    case DISK_DIRECT:
     case MEMORY:
       break;
     default:


[24/25] git commit: TEZ-1587. Some tez-examples fail in local mode (Contributed by Prakash Ramachandran)

Posted by je...@apache.org.
TEZ-1587. Some tez-examples fail in local mode (Contributed by Prakash Ramachandran)


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

Branch: refs/heads/TEZ-8
Commit: e328055e61a8dcaa1beed25e6da47f4a2279f3b6
Parents: 8bd2ca3
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Wed Sep 17 15:54:01 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Wed Sep 17 15:54:01 2014 +0530

----------------------------------------------------------------------
 CHANGES.txt                                       |  1 +
 .../java/org/apache/tez/examples/JoinExample.java |  4 ++--
 .../org/apache/tez/examples/JoinValidate.java     |  2 +-
 .../org/apache/tez/examples/OrderedWordCount.java |  4 ++--
 .../java/org/apache/tez/examples/WordCount.java   |  2 +-
 .../examples/BroadcastAndOneToOneExample.java     | 18 ++++++++++++++----
 .../tez/mapreduce/examples/FilterLinesByWord.java |  3 ++-
 .../examples/FilterLinesByWordOneToOne.java       |  3 ++-
 8 files changed, 25 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 83048bb..bd5569c 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -43,6 +43,7 @@ ALL CHANGES
   TEZ-1571. Add create method for DataSinkDescriptor.
   TEZ-1585. Memory leak in tez session mode.
   TEZ-1533. Request Events more often if a complete set of events is received by a task.
+  TEZ-1587. Some tez-examples fail in local mode.
 
 Release 0.5.0: 2014-09-03
 

http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
index 5ead275..3611fd6 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinExample.java
@@ -244,7 +244,7 @@ public class JoinExample extends Configured implements Tool {
     UnorderedPartitionedKVEdgeConfig streamConf =
         UnorderedPartitionedKVEdgeConfig
             .newBuilder(Text.class.getName(), NullWritable.class.getName(),
-                HashPartitioner.class.getName()).build();
+                HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
 
     /**
      * Connect the join vertex with the stream side
@@ -265,7 +265,7 @@ public class JoinExample extends Configured implements Tool {
        * value is null.
        */
       UnorderedKVEdgeConfig broadcastConf = UnorderedKVEdgeConfig.newBuilder(Text.class.getName(),
-          NullWritable.class.getName()).build();
+          NullWritable.class.getName()).setFromConfiguration(tezConf).build();
       hashSideEdgeProperty = broadcastConf.createDefaultBroadcastEdgeProperty();
     } else {
       /**

http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
index 3630184..e94df09 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/JoinValidate.java
@@ -186,7 +186,7 @@ public class JoinValidate extends Configured implements Tool {
     // better mechanism to configure the IOs.
     OrderedPartitionedKVEdgeConfig edgeConf = OrderedPartitionedKVEdgeConfig
         .newBuilder(Text.class.getName(), NullWritable.class.getName(),
-            HashPartitioner.class.getName()).build();
+            HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
 
     Vertex lhsVertex = Vertex.create(LHS_INPUT_NAME, ProcessorDescriptor.create(
         ForwardingProcessor.class.getName())).addDataSource("lhs",

http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java b/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
index fd18592..48884f4 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/OrderedWordCount.java
@@ -140,7 +140,7 @@ public class OrderedWordCount extends Configured implements Tool  {
     // Use Text key and IntWritable value to bring counts for each word in the same partition
     OrderedPartitionedKVEdgeConfig summationEdgeConf = OrderedPartitionedKVEdgeConfig
         .newBuilder(Text.class.getName(), IntWritable.class.getName(),
-            HashPartitioner.class.getName()).build();
+            HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
 
     // This vertex will be reading intermediate data via an input edge and writing intermediate data
     // via an output edge.
@@ -151,7 +151,7 @@ public class OrderedWordCount extends Configured implements Tool  {
     // partition. The data will be ordered by count and words grouped by count.
     OrderedPartitionedKVEdgeConfig sorterEdgeConf = OrderedPartitionedKVEdgeConfig
         .newBuilder(IntWritable.class.getName(), Text.class.getName(),
-            HashPartitioner.class.getName()).build();
+            HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
 
     // Use 1 task to bring all the data in one place for global sorted order. Essentially the number
     // of partitions is 1. So the NoOpSorter can be used to produce the globally ordered output

http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
----------------------------------------------------------------------
diff --git a/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java b/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
index e0e94cb..89eb68c 100644
--- a/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
+++ b/tez-examples/src/main/java/org/apache/tez/examples/WordCount.java
@@ -173,7 +173,7 @@ public class WordCount extends Configured implements Tool {
     // using this helper.
     OrderedPartitionedKVEdgeConfig edgeConf = OrderedPartitionedKVEdgeConfig
         .newBuilder(Text.class.getName(), IntWritable.class.getName(),
-            HashPartitioner.class.getName()).build();
+            HashPartitioner.class.getName()).setFromConfiguration(tezConf).build();
 
     // Create a vertex that reads the tokenized data and calculates the sum using the SumProcessor.
     // The number of tasks that do the work of this vertex depends on the number of partitions used 

http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
index 81f9581..e8a137c 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/BroadcastAndOneToOneExample.java
@@ -50,6 +50,7 @@ import org.apache.tez.runtime.api.ObjectRegistry;
 import org.apache.tez.runtime.api.ProcessorContext;
 import org.apache.tez.runtime.library.api.KeyValueReader;
 import org.apache.tez.runtime.library.api.KeyValueWriter;
+import org.apache.tez.runtime.library.api.TezRuntimeConfiguration;
 import org.apache.tez.runtime.library.conf.UnorderedKVEdgeConfig;
 import org.apache.tez.runtime.library.output.UnorderedKVOutput;
 import org.apache.tez.runtime.library.processor.SimpleProcessor;
@@ -159,7 +160,8 @@ public class BroadcastAndOneToOneExample extends Configured implements Tool {
         VertexManagerPluginDescriptor.create(InputReadyVertexManager.class.getName()));
 
     UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
-        .newBuilder(Text.class.getName(), IntWritable.class.getName()).build();
+        .newBuilder(Text.class.getName(), IntWritable.class.getName())
+        .setFromConfiguration(tezConf).build();
 
     DAG dag = DAG.create("BroadcastAndOneToOneExample");
     dag.addVertex(inputVertex)
@@ -188,9 +190,9 @@ public class BroadcastAndOneToOneExample extends Configured implements Tool {
 
     // staging dir
     FileSystem fs = FileSystem.get(tezConf);
-    String stagingDirStr = Path.SEPARATOR + "user" + Path.SEPARATOR
-        + user + Path.SEPARATOR+ ".staging" + Path.SEPARATOR
-        + Path.SEPARATOR + Long.toString(System.currentTimeMillis());    
+    String stagingDirStr = tezConf.get(TezConfiguration.TEZ_AM_STAGING_DIR,
+        TezConfiguration.TEZ_AM_STAGING_DIR_DEFAULT) + Path.SEPARATOR +
+        "BroadcastAndOneToOneExample" +  Path.SEPARATOR + Long.toString(System.currentTimeMillis());
     Path stagingDir = new Path(stagingDirStr);
     tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR, stagingDirStr);
     stagingDir = fs.makeQualified(stagingDir);
@@ -240,6 +242,14 @@ public class BroadcastAndOneToOneExample extends Configured implements Tool {
       printUsage();
       throw new TezException("Invalid command line");
     }
+
+    if (doLocalityCheck &&
+        getConf().getBoolean(TezConfiguration.TEZ_LOCAL_MODE,
+            TezConfiguration.TEZ_LOCAL_MODE_DEFAULT)) {
+      System.out.println("locality check is not valid in local mode. skipping");
+      doLocalityCheck = false;
+    }
+
     boolean status = run(getConf(), doLocalityCheck);
     return status ? 0 : 1;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
index 8430c68..f45b3d6 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
@@ -199,7 +199,8 @@ public class FilterLinesByWord extends Configured implements Tool {
     stage2Vertex.addDataSink("MROutput", DataSinkDescriptor.create(od, ocd, null));
 
     UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
-        .newBuilder(Text.class.getName(), TextLongPair.class.getName()).build();
+        .newBuilder(Text.class.getName(), TextLongPair.class.getName())
+        .setFromConfiguration(tezConf).build();
 
     DAG dag = DAG.create("FilterLinesByWord");
     Edge edge =

http://git-wip-us.apache.org/repos/asf/tez/blob/e328055e/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
index 515cf70..49cb30b 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
@@ -190,7 +190,8 @@ public class FilterLinesByWordOneToOne extends Configured implements Tool {
             OutputCommitterDescriptor.create(MROutputCommitter.class.getName()), null));
 
     UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
-        .newBuilder(Text.class.getName(), TextLongPair.class.getName()).build();
+        .newBuilder(Text.class.getName(), TextLongPair.class.getName())
+        .setFromConfiguration(tezConf).build();
 
     DAG dag = DAG.create("FilterLinesByWord");
     Edge edge =


[11/25] git commit: TEZ-1574. Support additional formats for the tez deployed archive. (sseth)

Posted by je...@apache.org.
TEZ-1574. Support additional formats for the tez deployed archive.
(sseth)


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

Branch: refs/heads/TEZ-8
Commit: d1ea44d0b4f53f729fc7842b5325d0f941b6dd1c
Parents: e709d62
Author: Siddharth Seth <ss...@apache.org>
Authored: Fri Sep 12 03:17:49 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Fri Sep 12 03:17:49 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                                    | 1 +
 .../src/main/java/org/apache/tez/client/TezClientUtils.java    | 6 +++++-
 2 files changed, 6 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/d1ea44d0/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4abc9bf..ff53e04 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -28,6 +28,7 @@ ALL CHANGES
   TEZ-1362. Remove DAG_COMPLETED in DAGEventType.
   TEZ-1519. TezTaskRunner should not initialize TezConfiguration in TezChild.
   TEZ-1534. Make client side configs available to AM and tasks.
+  TEZ-1574. Support additional formats for the tez deployed archive
 
 Release 0.5.0: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/d1ea44d0/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
index adda0a9..3297adc 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClientUtils.java
@@ -171,7 +171,11 @@ public class TezClientUtils {
      
       List<Path> tezJarPaths = Lists.newArrayListWithCapacity(tezJarUris.length);
 
-      if (tezJarUris.length == 1 && (tezJarUris[0].endsWith(".tar.gz") || tezJarUris[0].endsWith(".tgz"))) {
+      if (tezJarUris.length == 1 && (
+              tezJarUris[0].endsWith(".tar.gz") ||
+              tezJarUris[0].endsWith(".tgz") ||
+              tezJarUris[0].endsWith(".zip") ||
+              tezJarUris[0].endsWith(".tar"))) {
         String fileName = tezJarUris[0];
         if (fileName.endsWith(".tar.gz") || fileName.endsWith(".tgz")) {
           FileStatus fStatus = getLRFileStatus(fileName, conf, false)[0];


[15/25] git commit: TEZ-1569. Add tests for preemption (bikas)

Posted by je...@apache.org.
TEZ-1569. Add tests for preemption (bikas)


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

Branch: refs/heads/TEZ-8
Commit: 5e5683ab5e89f1fd88883ed40574c8fb04316dd1
Parents: 8e382b3
Author: Bikas Saha <bi...@apache.org>
Authored: Fri Sep 12 14:15:56 2014 -0700
Committer: Bikas Saha <bi...@apache.org>
Committed: Fri Sep 12 14:15:56 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   3 +-
 .../java/org/apache/tez/client/TezClient.java   |   2 +-
 .../java/org/apache/tez/client/LocalClient.java |  15 +-
 .../org/apache/tez/dag/app/DAGAppMaster.java    |  15 +-
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |   6 +
 .../apache/tez/dag/app/MockDAGAppMaster.java    | 245 +++++++++++++++++++
 .../org/apache/tez/dag/app/MockLocalClient.java |  48 ++++
 .../org/apache/tez/dag/app/MockTezClient.java   |  48 ++++
 .../org/apache/tez/dag/app/TestPreemption.java  | 203 +++++++++++++++
 9 files changed, 576 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 87729b3..73a3671 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -15,6 +15,7 @@ ALL CHANGES:
   TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized.
   TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts.
   TEZ-1578. Remove TeraSort from Tez codebase.
+  TEZ-1569. Add tests for preemption
 
 Release 0.5.1: Unreleased
 
@@ -33,7 +34,7 @@ ALL CHANGES
   TEZ-1534. Make client side configs available to AM and tasks.
   TEZ-1574. Support additional formats for the tez deployed archive
 
-Release 0.5.0: Unreleased
+Release 0.5.0: 2014-09-03
 
 INCOMPATIBLE CHANGES
   TEZ-1038. Move TaskLocationHint outside of VertexLocationHint.

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-api/src/main/java/org/apache/tez/client/TezClient.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/client/TezClient.java b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
index 13ca2dc..77ab20c 100644
--- a/tez-api/src/main/java/org/apache/tez/client/TezClient.java
+++ b/tez-api/src/main/java/org/apache/tez/client/TezClient.java
@@ -123,7 +123,7 @@ public class TezClient {
   }
 
   @Private
-  TezClient(String name, TezConfiguration tezConf, boolean isSession,
+  protected TezClient(String name, TezConfiguration tezConf, boolean isSession,
             @Nullable Map<String, LocalResource> localResources,
             @Nullable Credentials credentials) {
     this.clientName = name;

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
index 0b615fa..0a95cf0 100644
--- a/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
+++ b/tez-dag/src/main/java/org/apache/tez/client/LocalClient.java
@@ -30,6 +30,7 @@ import org.apache.hadoop.yarn.api.records.FinalApplicationStatus;
 import org.apache.hadoop.yarn.client.api.YarnClientApplication;
 import org.apache.hadoop.yarn.conf.YarnConfiguration;
 import org.apache.hadoop.yarn.exceptions.YarnException;
+import org.apache.hadoop.yarn.util.Clock;
 import org.apache.hadoop.yarn.util.Records;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
@@ -54,6 +55,8 @@ import org.apache.tez.dag.app.DAGAppMasterState;
 import org.apache.tez.dag.app.dag.DAG;
 import org.apache.tez.dag.utils.EnvironmentUpdateUtils;
 
+import com.google.common.annotations.VisibleForTesting;
+
 public class LocalClient extends FrameworkClient {
   public static final Logger LOG = Logger.getLogger(LocalClient.class);
 
@@ -286,8 +289,7 @@ public class LocalClient extends FrameworkClient {
           int nmHttpPort = YarnConfiguration.DEFAULT_NM_WEBAPP_PORT;
           long appSubmitTime = System.currentTimeMillis();
 
-          dagAppMaster =
-              new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+          dagAppMaster = createDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
                   new SystemClock(),
                   appSubmitTime, isSession, userDir.toUri().getPath());
           clientHandler = new DAGClientHandler(dagAppMaster);
@@ -305,4 +307,13 @@ public class LocalClient extends FrameworkClient {
 
     return thread;
   }
+  
+  // this can be overridden by test code to create a mock app
+  @VisibleForTesting
+  protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemptId,
+      ContainerId cId, String currentHost, int nmPort, int nmHttpPort,
+      Clock clock, long appSubmitTime, boolean isSession, String userDir) {
+    return new DAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+        new SystemClock(), appSubmitTime, isSession, userDir);
+  }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/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 9cd716a..ea0ab3b 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
@@ -156,12 +156,12 @@ import com.google.common.base.Function;
 import com.google.common.collect.Maps;
 
 /**
- * The Map-Reduce Application Master.
+ * The Tez DAG Application Master.
  * The state machine is encapsulated in the implementation of Job interface.
  * All state changes happens via Job interface. Each event
  * results in a Finite State Transition in Job.
  *
- * MR AppMaster is the composition of loosely coupled services. The services
+ * Tez DAG AppMaster is the composition of loosely coupled services. The services
  * interact with each other via events. The components resembles the
  * Actors model. The component acts on received event and send out the
  * events to other components.
@@ -443,6 +443,11 @@ public class DAGAppMaster extends AbstractService {
       System.exit(0);
     }
   }
+  
+  @VisibleForTesting
+  protected TaskSchedulerEventHandler getTaskSchedulerEventHandler() {
+    return taskSchedulerEventHandler;
+  }
 
   private synchronized void handle(DAGAppMasterEvent event) {
     switch (event.getType()) {
@@ -1434,14 +1439,14 @@ public class DAGAppMaster extends AbstractService {
     }
     return null;
   }
-
+  
   @Override
   public synchronized void serviceStart() throws Exception {
 
     //start all the components
     startServices();
     super.serviceStart();
-
+    
     // metrics system init is really init & start.
     // It's more test friendly to put it here.
     DefaultMetricsSystem.initialize("DAGAppMaster");
@@ -1882,7 +1887,7 @@ public class DAGAppMaster extends AbstractService {
 
     UserGroupInformation.setConfiguration(conf);
     Credentials credentials = UserGroupInformation.getCurrentUser().getCredentials();
-
+    
     appMaster.appMasterUgi = UserGroupInformation
         .createRemoteUser(jobUserName);
     appMaster.appMasterUgi.addCredentials(credentials);

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/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 daaa81b..23f9096 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
@@ -105,6 +105,7 @@ 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.records.TezDAGID;
+import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezVertexID;
 import org.apache.tez.dag.utils.TaskSpecificLaunchCmdOption;
 import org.apache.tez.dag.utils.RelocalizationUtils;
@@ -690,6 +691,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     }
     return vertex.getVertexStatus(statusOptions);
   }
+  
+  public TaskAttemptImpl getTaskAttempt(TezTaskAttemptID taId) {
+    return (TaskAttemptImpl) getVertex(taId.getTaskID().getVertexID()).getTask(taId.getTaskID())
+        .getAttempt(taId);
+  }
 
   protected void initializeVerticesAndStart() {
     for (Vertex v : vertices.values()) {

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
new file mode 100644
index 0000000..9fe9c4d
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockDAGAppMaster.java
@@ -0,0 +1,245 @@
+/**
+* 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;
+
+import java.io.IOException;
+import java.net.UnknownHostException;
+import java.util.Collections;
+import java.util.Map;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.service.AbstractService;
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerExitStatus;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.api.records.ContainerStatus;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.tez.common.ContainerContext;
+import org.apache.tez.common.ContainerTask;
+import org.apache.tez.dag.api.TezUncheckedException;
+import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
+import org.apache.tez.dag.app.launcher.ContainerLauncher;
+import org.apache.tez.dag.app.rm.NMCommunicatorEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorLaunchRequestEvent;
+import org.apache.tez.dag.app.rm.NMCommunicatorStopRequestEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEvent;
+import org.apache.tez.dag.app.rm.container.AMContainerEventLaunched;
+import org.apache.tez.dag.app.rm.container.AMContainerEventType;
+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.TaskAttemptCompletedEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.apache.tez.runtime.api.impl.EventMetaData.EventProducerConsumerType;
+
+import com.google.common.collect.Maps;
+
+@SuppressWarnings("unchecked")
+public class MockDAGAppMaster extends DAGAppMaster {
+  
+  MockContainerLauncher containerLauncher;
+  
+  // mock container launcher does not launch real tasks.
+  // Upon, launch of a container is simulates the container asking for tasks
+  // Upon receiving a task it simulates completion of the tasks
+  // It can be used to preempt the container for a given task
+  public class MockContainerLauncher extends AbstractService implements ContainerLauncher, Runnable {
+
+    BlockingQueue<NMCommunicatorEvent> eventQueue = new LinkedBlockingQueue<NMCommunicatorEvent>();
+    Thread eventHandlingThread;
+    
+    Map<ContainerId, ContainerData> containers = Maps.newConcurrentMap();
+    TaskAttemptListenerImpTezDag taListener;
+    
+    AtomicBoolean startScheduling = new AtomicBoolean(true);
+    AtomicBoolean goFlag;
+    
+    Map<TezTaskID, Integer> preemptedTasks = Maps.newConcurrentMap();
+    
+    public MockContainerLauncher(AtomicBoolean goFlag) {
+      super("MockContainerLauncher");
+      this.goFlag = goFlag;
+    }
+
+    public class ContainerData {
+      ContainerId cId;
+      TezTaskAttemptID taId;
+      String vName;
+      boolean completed;
+      
+      public ContainerData(ContainerId cId) {
+        this.cId = cId;
+      }
+      
+      void clear() {
+        taId = null;
+        vName = null;
+        completed = false;
+      }
+    }
+    
+    @Override
+    public void serviceStart() throws Exception {
+      taListener = (TaskAttemptListenerImpTezDag) getTaskAttemptListener();
+      eventHandlingThread = new Thread(this);
+      eventHandlingThread.start();
+    }
+
+    @Override
+    public void serviceStop() throws Exception {
+      if (eventHandlingThread != null) {
+        eventHandlingThread.interrupt();
+        eventHandlingThread.join(2000l);
+      }
+    }
+    
+    @Override
+    public void handle(NMCommunicatorEvent event) {
+      switch (event.getType()) {
+      case CONTAINER_LAUNCH_REQUEST:
+        launch((NMCommunicatorLaunchRequestEvent) event);
+        break;
+      case CONTAINER_STOP_REQUEST:
+        stop((NMCommunicatorStopRequestEvent)event);
+        break;
+      }
+    }
+    
+    
+    void waitToGo() {
+      synchronized (goFlag) {
+        goFlag.set(true);
+        goFlag.notify();
+        try {
+          goFlag.wait();
+        } catch (InterruptedException e) {
+          throw new TezUncheckedException(e);
+        }
+      }
+    }
+    
+    public void startScheduling(boolean value) {
+      startScheduling.set(value);
+    }
+
+    public Map<ContainerId, ContainerData> getContainers() {
+      return containers;
+    }
+    
+    public void preemptContainerForTask(TezTaskID tId, int uptoVersion) {
+      preemptedTasks.put(tId, uptoVersion);
+    }
+    
+    public void preemptContainer(ContainerData cData) {
+      getTaskSchedulerEventHandler().containerCompleted(null, 
+          ContainerStatus.newInstance(cData.cId, null, "Preempted", ContainerExitStatus.PREEMPTED));
+      cData.clear();
+    }
+    
+    void stop(NMCommunicatorStopRequestEvent event) {
+      // remove from simulated container list
+      containers.remove(event.getContainerId());
+      getContext().getEventHandler().handle(
+          new AMContainerEvent(event.getContainerId(), AMContainerEventType.C_NM_STOP_SENT));
+    }
+
+    void launch(NMCommunicatorLaunchRequestEvent event) {
+      // launch container by putting it in simulated container list
+      containers.put(event.getContainerId(), new ContainerData(event.getContainerId()));
+      getContext().getEventHandler().handle(new AMContainerEventLaunched(event.getContainerId()));      
+    }
+
+    @Override
+    public void run() {
+      // wait for test to sync with us and get a reference to us. Go when sync is done
+      waitToGo();
+      while(true) {
+        if (!startScheduling.get()) { // schedule when asked to do so by the test code
+          continue;
+        }
+        for (Map.Entry<ContainerId, ContainerData> entry : containers.entrySet()) {
+          ContainerData cData = entry.getValue();
+          ContainerId cId = entry.getKey();
+          if (cData.taId == null) {
+            // if container is not assigned a task, ask for a task
+            try {
+              ContainerTask cTask = taListener.getTask(new ContainerContext(cId.toString()));
+              if (cTask == null) {
+                continue;
+              }
+              if (cTask.shouldDie()) {
+                containers.remove(cId);
+              } else {
+                cData.taId = cTask.getTaskSpec().getTaskAttemptID();
+                cData.vName = cTask.getTaskSpec().getVertexName();
+              }
+            } catch (IOException e) {
+              e.printStackTrace();
+            }
+          } else if (!cData.completed) {
+            // container is assigned a task and task is not completed
+            // complete the task or preempt the task
+            Integer version = preemptedTasks.get(cData.taId.getTaskID()); 
+            if (version != null && cData.taId.getId() <= version.intValue()) {
+              preemptContainer(cData);
+            } else {
+              // send a done notification
+              TezVertexID vertexId = cData.taId.getTaskID().getVertexID();
+              cData.completed = true;
+              getContext().getEventHandler().handle(
+                  new VertexEventRouteEvent(vertexId, Collections.singletonList(new TezEvent(
+                      new TaskAttemptCompletedEvent(), new EventMetaData(
+                          EventProducerConsumerType.SYSTEM, cData.vName, "", cData.taId)))));
+              cData.clear();
+            }
+          }
+        }
+        try {
+          Thread.sleep(10);
+        } catch (InterruptedException e) {
+          System.out.println("Interrupted in mock container launcher thread");
+          break;
+        }
+      }
+    }
+    
+  }
+
+  public MockDAGAppMaster(ApplicationAttemptId applicationAttemptId, ContainerId containerId,
+      String nmHost, int nmPort, int nmHttpPort, Clock clock, long appSubmitTime,
+      boolean isSession, String workingDirectory, AtomicBoolean launcherGoFlag) {
+    super(applicationAttemptId, containerId, nmHost, nmPort, nmHttpPort, clock, appSubmitTime,
+        isSession, workingDirectory);
+    containerLauncher = new MockContainerLauncher(launcherGoFlag);
+  }
+  
+  // use mock container launcher for tests
+  @Override
+  protected ContainerLauncher createContainerLauncher(final AppContext context)
+      throws UnknownHostException {
+    return containerLauncher;
+  }
+  
+  public MockContainerLauncher getContainerLauncher() {
+    return containerLauncher;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
new file mode 100644
index 0000000..7e408e1
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockLocalClient.java
@@ -0,0 +1,48 @@
+/**
+* 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;
+
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.yarn.api.records.ApplicationAttemptId;
+import org.apache.hadoop.yarn.api.records.ContainerId;
+import org.apache.hadoop.yarn.util.Clock;
+import org.apache.hadoop.yarn.util.SystemClock;
+import org.apache.tez.client.LocalClient;
+
+public class MockLocalClient extends LocalClient {
+  MockDAGAppMaster mockApp;
+  AtomicBoolean mockAppLauncherGoFlag;
+  
+  public MockLocalClient(AtomicBoolean mockAppLauncherGoFlag) {
+    this.mockAppLauncherGoFlag = mockAppLauncherGoFlag;
+  }
+  
+  protected DAGAppMaster createDAGAppMaster(ApplicationAttemptId applicationAttemptId,
+      ContainerId cId, String currentHost, int nmPort, int nmHttpPort,
+      Clock clock, long appSubmitTime, boolean isSession, String userDir) {
+    mockApp = new MockDAGAppMaster(applicationAttemptId, cId, currentHost, nmPort, nmHttpPort,
+        new SystemClock(), appSubmitTime, isSession, userDir, mockAppLauncherGoFlag);
+    return mockApp;
+  }
+  
+  public MockDAGAppMaster getMockApp() {
+    return mockApp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java b/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
new file mode 100644
index 0000000..617415e
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/MockTezClient.java
@@ -0,0 +1,48 @@
+/**
+* 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;
+
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.security.Credentials;
+import org.apache.hadoop.yarn.api.records.LocalResource;
+import org.apache.tez.client.FrameworkClient;
+import org.apache.tez.client.TezClient;
+import org.apache.tez.dag.api.TezConfiguration;
+
+public class MockTezClient extends TezClient {
+  MockLocalClient client;
+  
+  MockTezClient(String name, TezConfiguration tezConf, boolean isSession,
+      Map<String, LocalResource> localResources, Credentials credentials,
+      AtomicBoolean mockAppLauncherGoFlag) {
+    super(name, tezConf, isSession, localResources, credentials);
+    this.client = new MockLocalClient(mockAppLauncherGoFlag);
+  }
+  
+  protected FrameworkClient createFrameworkClient() {
+    return client;
+  }
+  
+  public MockLocalClient getLocalClient() {
+    return client;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/5e5683ab/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
new file mode 100644
index 0000000..c7aacd4
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/TestPreemption.java
@@ -0,0 +1,203 @@
+/**
+* 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;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.tez.dag.api.DAG;
+import org.apache.tez.dag.api.Edge;
+import org.apache.tez.dag.api.EdgeProperty;
+import org.apache.tez.dag.api.EdgeProperty.DataMovementType;
+import org.apache.tez.dag.api.EdgeProperty.DataSourceType;
+import org.apache.tez.dag.api.EdgeProperty.SchedulingType;
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.OutputDescriptor;
+import org.apache.tez.dag.api.ProcessorDescriptor;
+import org.apache.tez.dag.api.TezConfiguration;
+import org.apache.tez.dag.api.Vertex;
+import org.apache.tez.dag.api.client.DAGClient;
+import org.apache.tez.dag.api.client.DAGStatus;
+import org.apache.tez.dag.app.MockDAGAppMaster.MockContainerLauncher;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+import org.apache.tez.dag.app.dag.impl.DAGImpl;
+import org.apache.tez.dag.app.dag.impl.TaskAttemptImpl;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.apache.tez.dag.records.TezVertexID;
+import org.junit.Assert;
+import org.junit.Test;
+
+@SuppressWarnings("deprecation")
+public class TestPreemption {
+  
+  static Configuration defaultConf;
+  static FileSystem localFs;
+  static Path workDir;
+  
+  static {
+    try {
+      defaultConf = new Configuration(false);
+      defaultConf.set("fs.defaultFS", "file:///");
+      defaultConf.setBoolean(TezConfiguration.TEZ_LOCAL_MODE, true);
+      localFs = FileSystem.getLocal(defaultConf);
+      workDir = new Path(new Path(System.getProperty("test.build.data", "/tmp")),
+          "TestDAGAppMaster").makeQualified(localFs);
+    } catch (IOException e) {
+      throw new RuntimeException("init failure", e);
+    }
+  }
+  
+  MockDAGAppMaster mockApp;    
+  MockContainerLauncher mockLauncher;
+  
+  int dagCount = 0;
+  
+  DAG createDAG(DataMovementType dmType) {
+    DAG dag = DAG.create("test-" + dagCount++);
+    Vertex vA = Vertex.create("A", ProcessorDescriptor.create("Proc.class"), 5);
+    Vertex vB = Vertex.create("B", ProcessorDescriptor.create("Proc.class"), 5);
+    Edge eAB = Edge.create(vA, vB, 
+    EdgeProperty.create(dmType, DataSourceType.PERSISTED,
+        SchedulingType.SEQUENTIAL, OutputDescriptor.create("O.class"),
+        InputDescriptor.create("I.class")));
+    
+    dag.addVertex(vA).addVertex(vB).addEdge(eAB);
+    return dag;
+  }
+  
+  @Test
+  public void testPreemptionWithoutSession() throws Exception {
+    System.out.println("TestPreemptionWithoutSession");
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+    AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false);
+    MockTezClient tezClient = new MockTezClient("testPreemption", tezconf, false, null, null,
+        mockAppLauncherGoFlag);
+    tezClient.start();
+    
+    DAGClient dagClient = tezClient.submitDAG(createDAG(DataMovementType.SCATTER_GATHER));
+    // now the MockApp has been started. sync with it to get the launcher
+    syncWithMockAppLauncher(false, mockAppLauncherGoFlag, tezClient);
+
+    DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
+    int vertexIndex = 0;
+    int upToTaskVersion = 3;
+    TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), vertexIndex);
+    TezTaskAttemptID taId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0);
+
+    mockLauncher.preemptContainerForTask(taId.getTaskID(), upToTaskVersion);
+    mockLauncher.startScheduling(true);
+    
+    dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+
+    for (int i=0; i<=upToTaskVersion; ++i) {
+      TezTaskAttemptID testTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), i);      
+      TaskAttemptImpl taImpl = dagImpl.getTaskAttempt(testTaId);
+      Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState());
+    }
+    
+    tezClient.stop();
+  }
+  
+  @Test
+  public void testPreemptionWithSession() throws Exception {
+    System.out.println("TestPreemptionWithSession");
+    MockTezClient tezClient = createTezSession();
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 0, "Scatter-Gather");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 0, "Scatter-Gather");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.BROADCAST), 0, "Broadcast");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.BROADCAST), 0, "Broadcast");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 0, "1-1");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 0, "1-1");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 1, "Scatter-Gather");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.SCATTER_GATHER), 1, "Scatter-Gather");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.BROADCAST), 1, "Broadcast");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.BROADCAST), 1, "Broadcast");
+    testPreemptionSingle(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 1, "1-1");
+    testPreemptionMultiple(tezClient, createDAG(DataMovementType.ONE_TO_ONE), 1, "1-1");
+    tezClient.stop();
+  }
+  
+  MockTezClient createTezSession() throws Exception {
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+    AtomicBoolean mockAppLauncherGoFlag = new AtomicBoolean(false);
+    MockTezClient tezClient = new MockTezClient("testPreemption", tezconf, true, null, null,
+        mockAppLauncherGoFlag);
+    tezClient.start();
+    syncWithMockAppLauncher(false, mockAppLauncherGoFlag, tezClient);
+    return tezClient;
+  }
+  
+  void syncWithMockAppLauncher(boolean allowScheduling, AtomicBoolean mockAppLauncherGoFlag, 
+      MockTezClient tezClient) throws Exception {
+    synchronized (mockAppLauncherGoFlag) {
+      while (!mockAppLauncherGoFlag.get()) {
+        mockAppLauncherGoFlag.wait();
+      }
+      mockApp = tezClient.getLocalClient().getMockApp();
+      mockLauncher = mockApp.getContainerLauncher();
+      mockLauncher.startScheduling(allowScheduling);
+      mockAppLauncherGoFlag.notify();
+    }     
+  }
+  
+  void testPreemptionSingle(MockTezClient tezClient, DAG dag, int vertexIndex, String info)
+      throws Exception {
+    testPreemptionJob(tezClient, dag, vertexIndex, 0, info + "-Single");
+  }
+
+  void testPreemptionMultiple(MockTezClient tezClient, DAG dag, int vertexIndex, String info)
+      throws Exception {
+    testPreemptionJob(tezClient, dag, vertexIndex, 3, info + "-Multiple");
+  }
+
+  void testPreemptionJob(MockTezClient tezClient, DAG dag, int vertexIndex,
+      int upToTaskVersion, String info) throws Exception {
+    System.out.println("TestPreemption - Running - " + info);
+    TezConfiguration tezconf = new TezConfiguration(defaultConf);
+    tezconf.setInt(TezConfiguration.TEZ_AM_TASK_MAX_FAILED_ATTEMPTS, 0);
+    
+    mockLauncher.startScheduling(false); // turn off scheduling to block DAG before submitting it
+    DAGClient dagClient = tezClient.submitDAG(dag);
+    
+    DAGImpl dagImpl = (DAGImpl) mockApp.getContext().getCurrentDAG();
+    TezVertexID vertexId = TezVertexID.getInstance(dagImpl.getID(), vertexIndex);
+    TezTaskAttemptID taId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), 0);
+
+    mockLauncher.preemptContainerForTask(taId.getTaskID(), upToTaskVersion);
+    mockLauncher.startScheduling(true);
+    
+    dagClient.waitForCompletion();
+    Assert.assertEquals(DAGStatus.State.SUCCEEDED, dagClient.getDAGStatus(null).getState());
+    
+    for (int i=0; i<=upToTaskVersion; ++i) {
+      TezTaskAttemptID testTaId = TezTaskAttemptID.getInstance(TezTaskID.getInstance(vertexId, 0), i);      
+      TaskAttemptImpl taImpl = dagImpl.getTaskAttempt(testTaId);
+      Assert.assertEquals(TaskAttemptStateInternal.KILLED, taImpl.getInternalState());
+    }
+    
+    System.out.println("TestPreemption - Done running - " + info);
+  }
+}


[07/25] git commit: TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized. (Jeff Zhang via hitesh)

Posted by je...@apache.org.
TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized. (Jeff Zhang via hitesh)


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

Branch: refs/heads/TEZ-8
Commit: fb05aace5062dc441da80f5ac8d4e322ab2a15ef
Parents: 16a0f57
Author: Hitesh Shah <hi...@apache.org>
Authored: Thu Sep 11 22:29:29 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Thu Sep 11 22:29:29 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/tez/dag/app/dag/impl/VertexImpl.java |  7 +-
 .../tez/dag/app/dag/impl/VertexManager.java     | 28 ++++----
 .../tez/dag/app/dag/impl/TestVertexManager.java | 76 ++++++++++++++++++++
 .../org/apache/tez/test/TestDAGRecovery.java    | 67 ++++++++++++++++-
 .../apache/tez/test/dag/MultiAttemptDAG.java    | 35 ++++++++-
 6 files changed, 195 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 519aaa6..f29d48d 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -11,6 +11,7 @@ ALL CHANGES:
   TEZ-1559. Add system tests for AM recovery.
   TEZ-850. Recovery unit tests.
   TEZ-853. Support counters recovery.
+  TEZ-1345. Add checks to guarantee all init events are written to recovery to consider vertex initialized.
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/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 31240cb..6437e5b 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
@@ -156,7 +156,6 @@ import org.apache.tez.runtime.api.impl.GroupInputSpec;
 import org.apache.tez.runtime.api.impl.InputSpec;
 import org.apache.tez.runtime.api.impl.OutputSpec;
 import org.apache.tez.runtime.api.impl.TezEvent;
-
 import com.google.common.annotations.VisibleForTesting;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.HashMultiset;
@@ -2782,10 +2781,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       VertexEventRootInputInitialized liInitEvent = (VertexEventRootInputInitialized) event;
       VertexState state = vertex.getState();
       if (state == VertexState.INITIALIZING) {
-        vertex.vertexManager.onRootVertexInitialized(
+        List<TezEvent> inputInfoEvents =
+            vertex.vertexManager.onRootVertexInitialized(
             liInitEvent.getInputName(),
             vertex.getAdditionalInputs().get(liInitEvent.getInputName())
                 .getIODescriptor(), liInitEvent.getEvents());
+        if (inputInfoEvents != null && !inputInfoEvents.isEmpty()) {
+          ROUTE_EVENT_TRANSITION.transition(vertex, new VertexEventRouteEvent(vertex.vertexId, inputInfoEvents));
+        }
       }
 
       vertex.numInitializedInputs++;

http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
index 177b946..c2ff660 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/VertexManager.java
@@ -24,6 +24,7 @@ import java.util.Collection;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
 
 import javax.annotation.Nullable;
 
@@ -44,7 +45,6 @@ import org.apache.tez.dag.app.AppContext;
 import org.apache.tez.dag.app.dag.Task;
 import org.apache.tez.dag.app.dag.TaskAttempt;
 import org.apache.tez.dag.app.dag.Vertex;
-import org.apache.tez.dag.app.dag.event.VertexEventRouteEvent;
 import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.apache.tez.runtime.api.Event;
@@ -68,10 +68,11 @@ public class VertexManager {
   VertexManagerPluginContextImpl pluginContext;
   UserPayload payload = null;
   AppContext appContext;
-    
+  ConcurrentHashMap<String, List<TezEvent>> cachedRootInputEventMap;
+
   class VertexManagerPluginContextImpl implements VertexManagerPluginContext {
     // TODO Add functionality to allow VertexManagers to send VertexManagerEvents
-    
+
     private EventMetaData rootEventSourceMetadata = new EventMetaData(EventProducerConsumerType.INPUT,
         managedVertex.getName(), "NULL_VERTEX", null);
     private Map<String, EventMetaData> destinationEventMetadataMap = Maps.newHashMap();
@@ -80,7 +81,7 @@ public class VertexManager {
     public Map<String, EdgeProperty> getInputVertexEdgeProperties() {
       // TODO Something similar for Initial Inputs - payload etc visible
       Map<Vertex, Edge> inputs = managedVertex.getInputVertices();
-      Map<String, EdgeProperty> vertexEdgeMap = 
+      Map<String, EdgeProperty> vertexEdgeMap =
                           Maps.newHashMapWithExpectedSize(inputs.size());
       for (Map.Entry<Vertex, Edge> entry : inputs.entrySet()) {
         vertexEdgeMap.put(entry.getKey().getName(), entry.getValue().getEdgeProperty());
@@ -115,7 +116,7 @@ public class VertexManager {
     @Override
     public Set<String> getVertexInputNames() {
       Set<String> inputNames = null;
-      Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+      Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
           inputs = managedVertex.getAdditionalInputs();
       if (inputs != null) {
         inputNames = inputs.keySet();
@@ -128,7 +129,6 @@ public class VertexManager {
       return payload;
     }
 
-    @SuppressWarnings("unchecked")
     @Override
     public void addRootInputEvents(final String inputName,
         Collection<InputDataInformationEvent> events) {
@@ -142,8 +142,8 @@ public class VertexManager {
               return tezEvent;
             }
           });
-      appContext.getEventHandler().handle(
-          new VertexEventRouteEvent(managedVertex.getVertexId(), Lists.newArrayList(tezEvents)));
+
+      cachedRootInputEventMap.put(inputName,Lists.newArrayList(tezEvents));
       // Recovery handling is taken care of by the Vertex.
     }
 
@@ -201,7 +201,7 @@ public class VertexManager {
     }
   }
 
-  public VertexManager(VertexManagerPluginDescriptor pluginDesc, 
+  public VertexManager(VertexManagerPluginDescriptor pluginDesc,
       Vertex managedVertex, AppContext appContext) {
     checkNotNull(pluginDesc, "pluginDesc is null");
     checkNotNull(managedVertex, "managedVertex is null");
@@ -209,12 +209,13 @@ public class VertexManager {
     this.pluginDesc = pluginDesc;
     this.managedVertex = managedVertex;
     this.appContext = appContext;
+    this.cachedRootInputEventMap = new ConcurrentHashMap<String, List<TezEvent>>();
   }
-  
+
   public VertexManagerPlugin getPlugin() {
     return plugin;
   }
-  
+
   public void initialize() {
     pluginContext = new VertexManagerPluginContextImpl();
     if (pluginDesc != null) {
@@ -246,7 +247,7 @@ public class VertexManager {
 
   public void onSourceTaskCompleted(TezTaskID tezTaskId) {
     Integer taskId = new Integer(tezTaskId.getId());
-    String vertexName = 
+    String vertexName =
         appContext.getCurrentDAG().getVertex(tezTaskId.getVertexID()).getName();
     plugin.onSourceTaskCompleted(vertexName, taskId);
   }
@@ -255,8 +256,9 @@ public class VertexManager {
     plugin.onVertexManagerEventReceived(vmEvent);
   }
 
-  public void onRootVertexInitialized(String inputName, 
+  public List<TezEvent> onRootVertexInitialized(String inputName,
       InputDescriptor inputDescriptor, List<Event> events) {
     plugin.onRootVertexInitialized(inputName, inputDescriptor, events);
+    return cachedRootInputEventMap.get(inputName);
   }
 }

http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
new file mode 100644
index 0000000..b3e66bc
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestVertexManager.java
@@ -0,0 +1,76 @@
+/**
+ * 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.RETURNS_DEEP_STUBS;
+import static org.mockito.Mockito.doReturn;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import java.util.LinkedList;
+import java.util.List;
+
+import org.apache.tez.dag.api.InputDescriptor;
+import org.apache.tez.dag.api.VertexManagerPluginDescriptor;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.dag.Vertex;
+import org.apache.tez.runtime.api.Event;
+import org.apache.tez.runtime.api.events.InputDataInformationEvent;
+import org.apache.tez.runtime.api.impl.TezEvent;
+import org.junit.Test;
+
+public class TestVertexManager {
+
+  @Test
+  public void testOnRootVertexInitialized() {
+    Vertex mockVertex = mock(Vertex.class, RETURNS_DEEP_STUBS);
+    AppContext mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+    doReturn("vertex1").when(mockVertex).getName();
+    when(
+        mockAppContext.getCurrentDAG().getVertex(any(String.class))
+            .getTotalTasks()).thenReturn(1);
+
+    VertexManager vm =
+        new VertexManager(
+            VertexManagerPluginDescriptor.create(RootInputVertexManager.class
+                .getName()), mockVertex, mockAppContext);
+    vm.initialize();
+    InputDescriptor id1 = mock(InputDescriptor.class);
+    List<Event> events1 = new LinkedList<Event>();
+    InputDataInformationEvent diEvent1 =
+        InputDataInformationEvent.createWithSerializedPayload(0, null);
+    events1.add(diEvent1);
+    List<TezEvent> tezEvents1 =
+        vm.onRootVertexInitialized("input1", id1, events1);
+    assertEquals(1, tezEvents1.size());
+    assertEquals(diEvent1, tezEvents1.get(0).getEvent());
+
+    InputDescriptor id2 = mock(InputDescriptor.class);
+    List<Event> events2 = new LinkedList<Event>();
+    InputDataInformationEvent diEvent2 =
+        InputDataInformationEvent.createWithSerializedPayload(0, null);
+    events2.add(diEvent2);
+    List<TezEvent> tezEvents2 =
+        vm.onRootVertexInitialized("input1", id2, events2);
+    assertEquals(tezEvents2.size(), 1);
+    assertEquals(diEvent2, tezEvents2.get(0).getEvent());
+  }
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/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 c9acdc2..7676313 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,21 +24,30 @@ 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.VertexDataMovementEventsGeneratedEvent;
+import org.apache.tez.dag.history.events.VertexInitializedEvent;
 import org.apache.tez.test.dag.MultiAttemptDAG;
 import org.apache.tez.test.dag.MultiAttemptDAG.FailingInputInitializer;
 import org.apache.tez.test.dag.MultiAttemptDAG.NoOpInput;
+import org.apache.tez.test.dag.MultiAttemptDAG.TestRootInputInitializer;
 import org.apache.tez.test.dag.SimpleVTestDAG;
 import org.junit.After;
 import org.junit.AfterClass;
@@ -48,6 +57,7 @@ import org.junit.BeforeClass;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.List;
 import java.util.Random;
 
 public class TestDAGRecovery {
@@ -61,6 +71,7 @@ public class TestDAGRecovery {
   private static MiniDFSCluster dfsCluster = null;
   private static TezClient tezSession = null;
   private static FileSystem remoteFs = null;
+  private static TezConfiguration tezConf = null;
 
   @BeforeClass
   public static void beforeClass() throws Exception {
@@ -120,7 +131,7 @@ public class TestDAGRecovery {
         .valueOf(new Random().nextInt(100000))));
     TezClientUtils.ensureStagingDirExists(conf, remoteStagingDir);
 
-    TezConfiguration tezConf = new TezConfiguration(miniTezCluster.getConfig());
+    tezConf = new TezConfiguration(miniTezCluster.getConfig());
     tezConf.setInt(TezConfiguration.DAG_RECOVERY_MAX_UNFLUSHED_EVENTS, 0);
     tezConf.set(TezConfiguration.TEZ_AM_LOG_LEVEL, "DEBUG");
     tezConf.set(TezConfiguration.TEZ_AM_STAGING_DIR,
@@ -130,6 +141,7 @@ public class TestDAGRecovery {
     tezConf.setInt(TezConfiguration.TEZ_AM_RESOURCE_MEMORY_MB, 500);
     tezConf.set(TezConfiguration.TEZ_AM_LAUNCH_CMD_OPTS, " -Xmx256m");
     tezConf.setBoolean(TezConfiguration.TEZ_AM_SESSION_MODE, true);
+    tezConf.set(TezConfiguration.TEZ_AM_STAGING_SCRATCH_DATA_AUTO_DELETE, "false");
 
     tezSession = TezClient.create("TestDAGRecovery", tezConf);
     tezSession.start();
@@ -165,13 +177,64 @@ 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);
+    for (int i=1; i<=3; ++i) {
+      Path currentAttemptRecoveryDataDir = TezCommonUtils.getAttemptRecoveryPath(recoveryDataDir,i);
+      Path recoveryFilePath = new Path(currentAttemptRecoveryDataDir,
+      appId.toString().replace("application", "dag") + "_1" + TezConstants.DAG_RECOVERY_RECOVER_FILE_SUFFIX);
+      List<HistoryEvent> historyEvents = 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) {
+          VertexDataMovementEventsGeneratedEvent dmEvent =
+              (VertexDataMovementEventsGeneratedEvent)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);
+    // add input to v1 to make sure that there will be init events for v1 (TEZ-1345)
+    DataSourceDescriptor dataSource =
+        DataSourceDescriptor.create(InputDescriptor.create(NoOpInput.class.getName()),
+           InputInitializerDescriptor.create(TestRootInputInitializer.class.getName()), null);
+    dag.getVertex("v1").addDataSource("Input", dataSource);
+
     runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
 
+    verifyRecoveryLog();
+
     // it should fail if submitting same dags in recovery mode (TEZ-1064)
-    try{
+    try {
       DAGClient dagClient = tezSession.submitDAG(dag);
       Assert.fail("Expected DAG submit to fail on duplicate dag name");
     } catch (TezException e) {

http://git-wip-us.apache.org/repos/asf/tez/blob/fb05aace/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 7fc9ad7..58b9413 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
@@ -22,6 +22,7 @@ import com.google.common.collect.Lists;
 import com.google.common.primitives.Ints;
 
 import java.nio.ByteBuffer;
+
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.conf.Configuration;
@@ -53,12 +54,15 @@ import org.apache.tez.runtime.api.OutputContext;
 import org.apache.tez.runtime.api.InputInitializer;
 import org.apache.tez.runtime.api.InputInitializerContext;
 import org.apache.tez.runtime.api.Writer;
+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.TezEvent;
 import org.apache.tez.test.TestInput;
 import org.apache.tez.test.TestOutput;
 import org.apache.tez.test.TestProcessor;
 
+import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -148,8 +152,15 @@ public class MultiAttemptDAG {
     }
 
     @Override
-    public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor, List<Event> events) {
-      // Do nothing
+    public void onRootVertexInitialized(String inputName,
+        InputDescriptor inputDescriptor, List<Event> events) {
+      List<InputDataInformationEvent> inputInfoEvents = new ArrayList<InputDataInformationEvent>();
+      for (Event event: events) {
+        if (event instanceof InputDataInformationEvent) {
+          inputInfoEvents.add((InputDataInformationEvent)event);
+        }
+      }
+      getContext().addRootInputEvents(inputName, inputInfoEvents);
     }
   }
 
@@ -214,6 +225,26 @@ public class MultiAttemptDAG {
     }
   }
 
+  public static class TestRootInputInitializer extends InputInitializer {
+
+    public TestRootInputInitializer(InputInitializerContext initializerContext) {
+      super(initializerContext);
+    }
+
+    @Override
+    public List<Event> initialize() throws Exception {
+      List<Event> events = new ArrayList<Event>();
+      events.add(InputDataInformationEvent.createWithSerializedPayload(0, ByteBuffer.allocate(0)));
+      return events;
+    }
+
+    @Override
+    public void handleInputInitializerEvent(List<InputInitializerEvent> events)
+        throws Exception {
+      throw new UnsupportedOperationException("Not supported");
+    }
+  }
+
   public static class FailingInputInitializer extends InputInitializer {
 
     public FailingInputInitializer(InputInitializerContext initializerContext) {


[17/25] git commit: TEZ-1524. Resolve user group information only if ACLs are enabled (gopalv)

Posted by je...@apache.org.
TEZ-1524. Resolve user group information only if ACLs are enabled (gopalv)


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

Branch: refs/heads/TEZ-8
Commit: edb841c08de123ff2c5ace0662ae78bf3c58f2c0
Parents: 9dd0cb4
Author: Gopal V <go...@apache.org>
Authored: Fri Sep 12 15:04:32 2014 -0700
Committer: Gopal V <go...@apache.org>
Committed: Fri Sep 12 15:04:32 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/tez/common/security/ACLManager.java  |  30 +-
 .../tez/common/security/TestACLManager.java     | 417 ++++++++++---------
 ...DAGClientAMProtocolBlockingPBServerImpl.java |  40 +-
 4 files changed, 250 insertions(+), 238 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index f71c2e2..59be260 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -17,6 +17,7 @@ ALL CHANGES:
   TEZ-1578. Remove TeraSort from Tez codebase.
   TEZ-1569. Add tests for preemption
   TEZ-1580. Change TestOrderedWordCount to optionally use MR configs.
+  TEZ-1524. Resolve user group information only if ACLs are enabled.
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
index d8be327..77ab065 100644
--- a/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
+++ b/tez-api/src/main/java/org/apache/tez/common/security/ACLManager.java
@@ -18,6 +18,7 @@
 
 package org.apache.tez.common.security;
 
+import java.util.Arrays;
 import java.util.Collection;
 import java.util.EnumSet;
 import java.util.HashMap;
@@ -29,6 +30,7 @@ import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.hadoop.classification.InterfaceAudience.Private;
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.tez.dag.api.TezConfiguration;
 
@@ -92,13 +94,19 @@ public class ACLManager {
   }
 
   @VisibleForTesting
-  boolean checkAccess(String user, Collection<String> userGroups, ACLType aclType) {
+  boolean checkAccess(UserGroupInformation ugi, ACLType aclType) {
+
     if (!aclsEnabled) {
       return true;
     }
+
+    String user = ugi.getShortUserName();
+    Collection<String> userGroups = Arrays.asList(ugi.getGroupNames());
+
     if (amUser.equals(user)) {
       return true;
     }
+
     if (EnumSet.of(ACLType.DAG_MODIFY_ACL, ACLType.DAG_VIEW_ACL).contains(aclType)) {
       if (dagUser != null && dagUser.equals(user)) {
         return true;
@@ -129,22 +137,22 @@ public class ACLManager {
     return false;
   }
 
-  public boolean checkAMViewAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_VIEW_ACL);
+  public boolean checkAMViewAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_VIEW_ACL);
   }
 
-  public boolean checkAMModifyAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_MODIFY_ACL);
+  public boolean checkAMModifyAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_MODIFY_ACL);
   }
 
-  public boolean checkDAGViewAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_VIEW_ACL)
-        || checkAccess(user, userGroups, ACLType.DAG_VIEW_ACL);
+  public boolean checkDAGViewAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_VIEW_ACL)
+        || checkAccess(ugi, ACLType.DAG_VIEW_ACL);
   }
 
-  public boolean checkDAGModifyAccess(String user, Collection<String> userGroups) {
-    return checkAccess(user, userGroups, ACLType.AM_MODIFY_ACL)
-        || checkAccess(user, userGroups, ACLType.DAG_MODIFY_ACL);
+  public boolean checkDAGModifyAccess(UserGroupInformation ugi) {
+    return checkAccess(ugi, ACLType.AM_MODIFY_ACL)
+        || checkAccess(ugi, ACLType.DAG_MODIFY_ACL);
   }
 
   public Map<ApplicationAccessType, String> toYARNACls() {

http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
index 7ad4ede..bc35b51 100644
--- a/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
+++ b/tez-api/src/test/java/org/apache/tez/common/security/TestACLManager.java
@@ -23,6 +23,7 @@ import java.util.Map;
 import java.util.Set;
 
 import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.hadoop.yarn.api.records.ApplicationAccessType;
 import org.apache.tez.dag.api.TezConfiguration;
 import org.apache.tez.dag.api.TezConstants;
@@ -33,138 +34,141 @@ import com.google.common.collect.Sets;
 
 public class TestACLManager {
 
-  private static final Set<String> noGroups = Sets.newHashSet();
+  private static final String[] noGroups = new String[0];
 
   @Test
   public void testCurrentUserACLChecks() {
-    String currentUser = "currentUser";
-    ACLManager aclManager = new ACLManager(currentUser);
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation dagUser = UserGroupInformation.createUserForTesting("dagUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", noGroups);
 
-    String user = "user1";
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName());
+
+    UserGroupInformation user = user1;
+
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
     user = currentUser;
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
-    aclManager = new ACLManager(currentUser, new Configuration(false));
+    aclManager = new ACLManager(currentUser.getShortUserName(), new Configuration(false));
 
-    user = "user1";
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    user = user1;
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
     user = currentUser;
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
 
-    String dagUser = "dagUser";
-    ACLManager dagAclManager = new ACLManager(aclManager, dagUser, new Configuration(false));
+    ACLManager dagAclManager = new ACLManager(aclManager, dagUser.getShortUserName(), new Configuration(false));
     user = dagUser;
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(dagAclManager.checkAccess(user, null, ACLType.DAG_VIEW_ACL));
-    Assert.assertTrue(dagAclManager.checkAccess(user, null, ACLType.DAG_MODIFY_ACL));
-    user = "user1";
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.DAG_VIEW_ACL));
-    Assert.assertFalse(dagAclManager.checkAccess(user, null, ACLType.DAG_MODIFY_ACL));
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL));
+    Assert.assertTrue(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL));
+    user = user1;
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_VIEW_ACL));
+    Assert.assertFalse(dagAclManager.checkAccess(user, ACLType.DAG_MODIFY_ACL));
   }
 
   @Test
   public void testOtherUserACLChecks() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
+
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4, grp3, grp4.
-    String viewACLs = user1 + "," + user4
+    String viewACLs = user1.getShortUserName() + "," + user4.getShortUserName()
         + "   " + "grp3,grp4  ";
     // Modify ACLs: user3, grp6, grp7
-    String modifyACLs = user3 + "  " + "grp6,grp7";
+    String modifyACLs = user3.getShortUserName() + "  " + "grp6,grp7";
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager(currentUser, conf);
-
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user1, groups1, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user2, groups2, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user3, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user4, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_VIEW_ACL));
-
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user1, groups1, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user3, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user4, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user5, groups3, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_MODIFY_ACL));
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
+
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user5,  ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL));
+
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL));
   }
 
   @Test
   public void testNoGroupsACLChecks() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
+
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4
-    String viewACLs = user1 + "," + user4 + " ";
+    String viewACLs = user1.getShortUserName() + "," + user4.getShortUserName() + " ";
     // Modify ACLs: user3
-    String modifyACLs = "user3  ";
+    String modifyACLs = user3.getShortUserName() + " ";
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager(currentUser, conf);
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user1, groups1, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user3, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user4, noGroups, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_VIEW_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_VIEW_ACL));
-
-    Assert.assertTrue(aclManager.checkAccess(currentUser, null, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user1, groups1, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user2, groups2, ACLType.AM_MODIFY_ACL));
-    Assert.assertTrue(aclManager.checkAccess(user3, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user4, noGroups, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user5, groups3, ACLType.AM_MODIFY_ACL));
-    Assert.assertFalse(aclManager.checkAccess(user6, noGroups, ACLType.AM_MODIFY_ACL));
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user1, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user3, ACLType.AM_VIEW_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user4, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_VIEW_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_VIEW_ACL));
+
+    Assert.assertTrue(aclManager.checkAccess(currentUser, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user1, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user2, ACLType.AM_MODIFY_ACL));
+    Assert.assertTrue(aclManager.checkAccess(user3, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user4, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user5, ACLType.AM_MODIFY_ACL));
+    Assert.assertFalse(aclManager.checkAccess(user6, ACLType.AM_MODIFY_ACL));
   }
 
   @Test
   public void checkAMACLs() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
+
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4, grp3, grp4.
@@ -174,55 +178,55 @@ public class TestACLManager {
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager(currentUser, conf);
-
-    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user4, noGroups));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user6, noGroups));
-
-    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user6, noGroups));
-
-    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user4, noGroups));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user6, noGroups));
-
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user6, noGroups));
+    ACLManager aclManager = new ACLManager(currentUser.getShortUserName(), conf);
+
+    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user2));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user3));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user4));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user5));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user6));
+
+    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user5));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user6));
+
+    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user2));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user3));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user4));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user5));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user6));
+
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user6));
 
   }
 
   @Test
   public void checkDAGACLs() throws IOException {
-    Set<String> groups1 = Sets.newHashSet("grp1", "grp2");
-    Set<String> groups2 = Sets.newHashSet("grp3", "grp4");
-    Set<String> groups3 = Sets.newHashSet("grp5", "grp6");
-
-    String currentUser = "currentUser";
-    String user1 = "user1"; // belongs to grp1 and grp2
-    String user2 = "user2"; // belongs to grp3 and grp4
-    String user3 = "user3";
-    String user4 = "user4";
-    String user5 = "user5"; // belongs to grp5 and grp6
-    String user6 = "user6";
+    String[] groups1 = new String[] {"grp1", "grp2"};
+    String[] groups2 = new String[] {"grp3", "grp4"};
+    String[] groups3 = new String[] {"grp5", "grp6"};
+
+    UserGroupInformation currentUser = UserGroupInformation.createUserForTesting("currentUser", noGroups);
+    UserGroupInformation user1 = UserGroupInformation.createUserForTesting("user1", groups1); // belongs to grp1 and grp2
+    UserGroupInformation user2 = UserGroupInformation.createUserForTesting("user2", groups2); // belongs to grp3 and grp4
+    UserGroupInformation user3 = UserGroupInformation.createUserForTesting("user3", noGroups);
+    UserGroupInformation user4 = UserGroupInformation.createUserForTesting("user4", noGroups);
+    UserGroupInformation user5 = UserGroupInformation.createUserForTesting("user5", groups3); // belongs to grp5 and grp6
+    UserGroupInformation user6 = UserGroupInformation.createUserForTesting("user6", noGroups);
 
     Configuration conf = new Configuration(false);
     // View ACLs: user1, user4, grp3, grp4.
@@ -239,46 +243,46 @@ public class TestACLManager {
     conf.set(TezConstants.TEZ_DAG_VIEW_ACLS, dagViewACLs);
     conf.set(TezConstants.TEZ_DAG_MODIFY_ACLS, dagModifyACLs);
 
-    String dagUser = "dagUser";
-
-    ACLManager amAclManager = new ACLManager(currentUser, conf);
-    ACLManager aclManager = new ACLManager(amAclManager, dagUser, conf);
-
-    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkAMViewAccess(dagUser, null));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkAMViewAccess(user4, noGroups));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMViewAccess(user6, noGroups));
-
-    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser, null));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(dagUser, null));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkAMModifyAccess(user5, groups3));
-    Assert.assertFalse(aclManager.checkAMModifyAccess(user6, noGroups));
-
-    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(dagUser, null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user1, groups1));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user2, groups2));
-    Assert.assertFalse(aclManager.checkDAGViewAccess(user3, noGroups));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user5, groups3));
-    Assert.assertTrue(aclManager.checkDAGViewAccess(user6, noGroups));
-
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser, null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(dagUser, null));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1, groups1));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2, groups2));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3, noGroups));
-    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4, noGroups));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5, groups3));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess(user6, noGroups));
+    UserGroupInformation dagUser = UserGroupInformation.createUserForTesting("dagUser", noGroups);
+
+    ACLManager amAclManager = new ACLManager(currentUser.getShortUserName(), conf);
+    ACLManager aclManager = new ACLManager(amAclManager, dagUser.getShortUserName(), conf);
+
+    Assert.assertTrue(aclManager.checkAMViewAccess(currentUser));
+    Assert.assertFalse(aclManager.checkAMViewAccess(dagUser));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user2));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user3));
+    Assert.assertTrue(aclManager.checkAMViewAccess(user4));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user5));
+    Assert.assertFalse(aclManager.checkAMViewAccess(user6));
+
+    Assert.assertTrue(aclManager.checkAMModifyAccess(currentUser));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(dagUser));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(user5));
+    Assert.assertFalse(aclManager.checkAMModifyAccess(user6));
+
+    Assert.assertTrue(aclManager.checkDAGViewAccess(currentUser));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(dagUser));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user2));
+    Assert.assertFalse(aclManager.checkDAGViewAccess(user3));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user4));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user5));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(user6));
+
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(currentUser));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(dagUser));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user1));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user2));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user3));
+    Assert.assertFalse(aclManager.checkDAGModifyAccess(user4));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user5));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(user6));
 
   }
 
@@ -290,15 +294,18 @@ public class TestACLManager {
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
 
-    ACLManager aclManager = new ACLManager("a1", conf);
-    Assert.assertTrue(aclManager.checkAMViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("u1", null));
+    UserGroupInformation a1 = UserGroupInformation.createUserForTesting("a1", noGroups);
+    UserGroupInformation u1 = UserGroupInformation.createUserForTesting("u1", noGroups);
+
+    ACLManager aclManager = new ACLManager(a1.getShortUserName(), conf);
+    Assert.assertTrue(aclManager.checkAMViewAccess(a1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(u1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(u1));
   }
 
   @Test
@@ -309,25 +316,29 @@ public class TestACLManager {
     String modifyACLs = "a2,u2 ";
     conf.set(TezConfiguration.TEZ_AM_VIEW_ACLS, viewACLs);
     conf.set(TezConfiguration.TEZ_AM_MODIFY_ACLS, modifyACLs);
-    ACLManager aclManager = new ACLManager("a1", conf);
-    Assert.assertTrue(aclManager.checkAMViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkAMModifyAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGViewAccess("u1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("a1", null));
-    Assert.assertTrue(aclManager.checkDAGModifyAccess("u1", null));
+
+    UserGroupInformation a1 = UserGroupInformation.createUserForTesting("a1", noGroups);
+    UserGroupInformation u1 = UserGroupInformation.createUserForTesting("u1", noGroups);
+
+    ACLManager aclManager = new ACLManager(a1.getShortUserName(), conf);
+    Assert.assertTrue(aclManager.checkAMViewAccess(a1));
+    Assert.assertTrue(aclManager.checkAMViewAccess(u1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkAMModifyAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGViewAccess(u1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(a1));
+    Assert.assertTrue(aclManager.checkDAGModifyAccess(u1));
 
     ACLManager dagAclManager = new ACLManager(aclManager, "dagUser", null);
-    Assert.assertTrue(dagAclManager.checkAMViewAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkAMViewAccess("u1", null));
-    Assert.assertTrue(dagAclManager.checkAMModifyAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkAMModifyAccess("u1", null));
-    Assert.assertTrue(dagAclManager.checkDAGViewAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkDAGViewAccess("u1", null));
-    Assert.assertTrue(dagAclManager.checkDAGModifyAccess("a1", null));
-    Assert.assertTrue(dagAclManager.checkDAGModifyAccess("u1", null));
+    Assert.assertTrue(dagAclManager.checkAMViewAccess(a1));
+    Assert.assertTrue(dagAclManager.checkAMViewAccess(u1));
+    Assert.assertTrue(dagAclManager.checkAMModifyAccess(a1));
+    Assert.assertTrue(dagAclManager.checkAMModifyAccess(u1));
+    Assert.assertTrue(dagAclManager.checkDAGViewAccess(a1));
+    Assert.assertTrue(dagAclManager.checkDAGViewAccess(u1));
+    Assert.assertTrue(dagAclManager.checkDAGModifyAccess(a1));
+    Assert.assertTrue(dagAclManager.checkDAGModifyAccess(u1));
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/tez/blob/edb841c0/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
index 6381b71..c054305 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/api/client/rpc/DAGClientAMProtocolBlockingPBServerImpl.java
@@ -62,17 +62,9 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
     this.real = real;
   }
 
-  private String getRPCUserName() throws ServiceException {
+  private UserGroupInformation getRPCUser() throws ServiceException {
     try {
-      return UserGroupInformation.getCurrentUser().getShortUserName();
-    } catch (IOException e) {
-      throw wrapException(e);
-    }
-  }
-
-  private List<String> getRPCUserGroups() throws ServiceException {
-    try {
-      return Arrays.asList(UserGroupInformation.getCurrentUser().getGroupNames());
+      return UserGroupInformation.getCurrentUser();
     } catch (IOException e) {
       throw wrapException(e);
     }
@@ -81,8 +73,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
   @Override
   public GetAllDAGsResponseProto getAllDAGs(RpcController controller,
       GetAllDAGsRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMViewAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMViewAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM view operation");
     }
     try{
@@ -96,10 +88,10 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
   @Override
   public GetDAGStatusResponseProto getDAGStatus(RpcController controller,
       GetDAGStatusRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
+    UserGroupInformation user = getRPCUser();
     try {
       String dagId = request.getDagId();
-      if (!real.getACLManager(dagId).checkDAGViewAccess(user, getRPCUserGroups())) {
+      if (!real.getACLManager(dagId).checkDAGViewAccess(user)) {
         throw new AccessControlException("User " + user + " cannot perform DAG view operation");
       }
       DAGStatus status;
@@ -118,10 +110,10 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
   @Override
   public GetVertexStatusResponseProto getVertexStatus(RpcController controller,
       GetVertexStatusRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
+    UserGroupInformation user = getRPCUser();
     try {
       String dagId = request.getDagId();
-      if (!real.getACLManager(dagId).checkDAGViewAccess(user, getRPCUserGroups())) {
+      if (!real.getACLManager(dagId).checkDAGViewAccess(user)) {
         throw new AccessControlException("User " + user + " cannot perform DAG view operation");
       }
       String vertexName = request.getVertexName();
@@ -140,10 +132,10 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
   @Override
   public TryKillDAGResponseProto tryKillDAG(RpcController controller,
       TryKillDAGRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
+    UserGroupInformation user = getRPCUser();
     try {
       String dagId = request.getDagId();
-      if (!real.getACLManager(dagId).checkDAGModifyAccess(user, getRPCUserGroups())) {
+      if (!real.getACLManager(dagId).checkDAGModifyAccess(user)) {
         throw new AccessControlException("User " + user + " cannot perform DAG modify operation");
       }
       real.tryKillDAG(dagId);
@@ -156,8 +148,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
   @Override
   public SubmitDAGResponseProto submitDAG(RpcController controller,
       SubmitDAGRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMModifyAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMModifyAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM modify operation");
     }
     try{
@@ -181,8 +173,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
   @Override
   public ShutdownSessionResponseProto shutdownSession(RpcController arg0,
       ShutdownSessionRequestProto arg1) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMModifyAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMModifyAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM modify operation");
     }
     real.shutdownAM();
@@ -192,8 +184,8 @@ public class DAGClientAMProtocolBlockingPBServerImpl implements DAGClientAMProto
   @Override
   public GetAMStatusResponseProto getAMStatus(RpcController controller,
       GetAMStatusRequestProto request) throws ServiceException {
-    String user = getRPCUserName();
-    if (!real.getACLManager().checkAMViewAccess(user, getRPCUserGroups())) {
+    UserGroupInformation user = getRPCUser();
+    if (!real.getACLManager().checkAMViewAccess(user)) {
       throw new AccessControlException("User " + user + " cannot perform AM view operation");
     }
     try {


[22/25] git commit: Update CHANGES.txt related to TEZ-1585

Posted by je...@apache.org.
Update CHANGES.txt related to TEZ-1585


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

Branch: refs/heads/TEZ-8
Commit: 3b34c4182839c63a0fa0dac66b22d8990acb04e0
Parents: 938097d
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Tue Sep 16 12:10:58 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Tue Sep 16 12:10:58 2014 +0530

----------------------------------------------------------------------
 CHANGES.txt | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/3b34c418/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 97cf79b..ca4e7e2 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -41,6 +41,7 @@ ALL CHANGES
   TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
   of DAG submission
   TEZ-1571. Add create method for DataSinkDescriptor.
+  TEZ-1585. Memory leak in tez session mode.
 
 Release 0.5.0: 2014-09-03
 


[04/25] git commit: TEZ-1543. Shuffle Errors on heavy load (causing task retries) (Rajesh Balamohan)

Posted by je...@apache.org.
TEZ-1543. Shuffle Errors on heavy load (causing task retries)  (Rajesh Balamohan)


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

Branch: refs/heads/TEZ-8
Commit: bf6ac4e7811c2ea5a327cac7d8a109e3c57fd8e7
Parents: 80b5795
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Fri Sep 12 03:57:40 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Sep 12 03:57:40 2014 +0530

----------------------------------------------------------------------
 .../tez/runtime/library/shuffle/common/HttpConnection.java       | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/bf6ac4e7/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
----------------------------------------------------------------------
diff --git a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
index 7208c16..d664f88 100644
--- a/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
+++ b/tez-runtime-library/src/main/java/org/apache/tez/runtime/library/shuffle/common/HttpConnection.java
@@ -160,7 +160,7 @@ public class HttpConnection {
         break;
       } catch (IOException ioe) {
         // Don't attempt another connect if already cleanedup.
-        if (!cleanup) {
+        if (cleanup) {
           LOG.info("Cleanup is set to true. Not attempting to"
               + " connect again. Last exception was: ["
               + ioe.getClass().getName() + ", " + ioe.getMessage() + "]");
@@ -170,7 +170,7 @@ public class HttpConnection {
         connectionTimeout -= unit;
         // throw an exception if we have waited for timeout amount of time
         // note that the updated value if timeout is used here
-        if (connectionTimeout == 0) {
+        if (connectionTimeout <= 0) {
           throw ioe;
         }
         // reset the connect timeout for the last try


[05/25] git commit: TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput() (Rajesh Balamohan)

Posted by je...@apache.org.
TEZ-1494. DAG hangs waiting for ShuffleManager.getNextInput() (Rajesh Balamohan)


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

Branch: refs/heads/TEZ-8
Commit: fcc74261a0d76d00580f4d0dca042a1ed014ccec
Parents: bf6ac4e
Author: Rajesh Balamohan <rb...@apache.org>
Authored: Fri Sep 12 04:09:21 2014 +0530
Committer: Rajesh Balamohan <rb...@apache.org>
Committed: Fri Sep 12 04:09:21 2014 +0530

----------------------------------------------------------------------
 .../dag/impl/ImmediateStartVertexManager.java   |  92 ++++-
 .../app/dag/impl/RootInputVertexManager.java    |  27 +-
 .../tez/dag/app/dag/impl/TestVertexImpl.java    | 364 +++++++++++++++++--
 3 files changed, 417 insertions(+), 66 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/fcc74261/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
index b202d70..ac2b851 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/ImmediateStartVertexManager.java
@@ -18,9 +18,11 @@
 
 package org.apache.tez.dag.app.dag.impl;
 
-import java.util.List;
-import java.util.Map;
-
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.tez.dag.api.EdgeProperty;
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.VertexManagerPlugin;
 import org.apache.tez.dag.api.VertexManagerPluginContext;
@@ -28,29 +30,101 @@ import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
 import org.apache.tez.runtime.api.Event;
 import org.apache.tez.runtime.api.events.VertexManagerEvent;
 
-import com.google.common.collect.Lists;
+import java.util.List;
+import java.util.Map;
 
 /**
  * Starts all tasks immediately on vertex start
  */
 public class ImmediateStartVertexManager extends VertexManagerPlugin {
 
+  private static final Log LOG = LogFactory.getLog(ImmediateStartVertexManager.class);
+
+  private final Map<String, SourceVertexInfo> srcVertexInfo = Maps.newHashMap();
+  private int managedTasks;
+  private boolean tasksScheduled = false;
+
+  class SourceVertexInfo {
+    EdgeProperty edgeProperty;
+    int numFinishedTasks;
+
+    SourceVertexInfo(EdgeProperty edgeProperty) {
+      this.edgeProperty = edgeProperty;
+    }
+  }
+
   public ImmediateStartVertexManager(VertexManagerPluginContext context) {
     super(context);
   }
 
   @Override
   public void onVertexStarted(Map<String, List<Integer>> completions) {
-    int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
-    List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
-    for (int i=0; i<numTasks; ++i) {
-      scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
+    managedTasks = getContext().getVertexNumTasks(getContext().getVertexName());
+    Map<String, EdgeProperty> edges = getContext().getInputVertexEdgeProperties();
+    for (Map.Entry<String, EdgeProperty> entry : edges.entrySet()) {
+      String srcVertex = entry.getKey();
+      EdgeProperty edgeProp = entry.getValue();
+      srcVertexInfo.put(srcVertex, new SourceVertexInfo(edgeProp));
+    }
+
+    //handle completions
+    for (Map.Entry<String, List<Integer>> entry : completions.entrySet()) {
+      for (Integer task : entry.getValue()) {
+        handleSourceTaskFinished(entry.getKey(), task);
+      }
+    }
+    scheduleTasks();
+  }
+
+  private void handleSourceTaskFinished(String vertex, Integer taskId) {
+    SourceVertexInfo srcInfo = srcVertexInfo.get(vertex);
+    //Not mandatory to check for duplicate completions here
+    srcInfo.numFinishedTasks++;
+  }
+
+  private void scheduleTasks() {
+    if (!canScheduleTasks()) {
+      return;
+    }
+
+    List<TaskWithLocationHint> tasksToStart = Lists.newArrayListWithCapacity(managedTasks);
+    for (int i = 0; i < managedTasks; ++i) {
+      tasksToStart.add(new TaskWithLocationHint(new Integer(i), null));
+    }
+
+    if (!tasksToStart.isEmpty()) {
+      LOG.info("Starting " + tasksToStart.size() + " in " + getContext().getVertexName());
+      getContext().scheduleVertexTasks(tasksToStart);
     }
-    getContext().scheduleVertexTasks(scheduledTasks);
+    tasksScheduled = true;
+  }
+
+  private boolean canScheduleTasks() {
+    //Check if at least 1 task is finished from each source vertex (in case of broadcast &
+    // one-to-one or custom)
+    for (Map.Entry<String, SourceVertexInfo> entry : srcVertexInfo.entrySet()) {
+      SourceVertexInfo srcVertexInfo = entry.getValue();
+      switch(srcVertexInfo.edgeProperty.getDataMovementType()) {
+      case ONE_TO_ONE:
+      case BROADCAST:
+      case CUSTOM:
+        if (srcVertexInfo.numFinishedTasks == 0) {
+          //do not schedule tasks until a task from source task is complete
+          return false;
+        }
+      default:
+        break;
+      }
+    }
+    return true;
   }
 
   @Override
   public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
+    handleSourceTaskFinished(srcVertexName, attemptId);
+    if (!tasksScheduled) {
+      scheduleTasks();
+    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/fcc74261/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
index e6ffdc5..e850286 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/app/dag/impl/RootInputVertexManager.java
@@ -18,27 +18,23 @@
 
 package org.apache.tez.dag.app.dag.impl;
 
-import java.nio.ByteBuffer;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.tez.dag.api.InputDescriptor;
 import org.apache.tez.dag.api.UserPayload;
-import org.apache.tez.dag.api.VertexManagerPlugin;
 import org.apache.tez.dag.api.VertexManagerPluginContext;
-import org.apache.tez.dag.api.VertexManagerPluginContext.TaskWithLocationHint;
 import org.apache.tez.runtime.api.Event;
 import org.apache.tez.runtime.api.InputSpecUpdate;
 import org.apache.tez.runtime.api.events.InputConfigureVertexTasksEvent;
 import org.apache.tez.runtime.api.events.InputDataInformationEvent;
 import org.apache.tez.runtime.api.events.InputUpdatePayloadEvent;
-import org.apache.tez.runtime.api.events.VertexManagerEvent;
 
 import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
 
-public class RootInputVertexManager extends VertexManagerPlugin {
+public class RootInputVertexManager extends ImmediateStartVertexManager {
 
   private String configuredInputName;
 
@@ -46,27 +42,6 @@ public class RootInputVertexManager extends VertexManagerPlugin {
     super(context);
   }
 
-  @Override
-  public void initialize() {
-  }
-
-  @Override
-  public void onVertexStarted(Map<String, List<Integer>> completions) {
-    int numTasks = getContext().getVertexNumTasks(getContext().getVertexName());
-    List<TaskWithLocationHint> scheduledTasks = Lists.newArrayListWithCapacity(numTasks);
-    for (int i=0; i<numTasks; ++i) {
-      scheduledTasks.add(new TaskWithLocationHint(new Integer(i), null));
-    }
-    getContext().scheduleVertexTasks(scheduledTasks);
-  }
-
-  @Override
-  public void onSourceTaskCompleted(String srcVertexName, Integer attemptId) {
-  }
-
-  @Override
-  public void onVertexManagerEventReceived(VertexManagerEvent vmEvent) {
-  }
 
   @Override
   public void onRootVertexInitialized(String inputName, InputDescriptor inputDescriptor,

http://git-wip-us.apache.org/repos/asf/tez/blob/fcc74261/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 d894928..04e2219 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
@@ -20,6 +20,7 @@ package org.apache.tez.dag.app.dag.impl;
 
 import java.nio.ByteBuffer;
 
+import static org.junit.Assert.assertTrue;
 import static org.mockito.Matchers.any;
 import static org.mockito.Matchers.anyInt;
 import static org.mockito.Mockito.doReturn;
@@ -1879,39 +1880,39 @@ public class TestVertexImpl {
     Assert.assertEquals(2, v3.getOutputVerticesCount());
     Assert.assertEquals(2, v3.getOutputVerticesCount());
 
-    Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
+    assertTrue("vertex1".equals(v3.getInputSpecList(0).get(0)
         .getSourceVertexName())
         || "vertex2".equals(v3.getInputSpecList(0).get(0)
-            .getSourceVertexName()));
-    Assert.assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
+        .getSourceVertexName()));
+    assertTrue("vertex1".equals(v3.getInputSpecList(0).get(1)
         .getSourceVertexName())
         || "vertex2".equals(v3.getInputSpecList(0).get(1)
-            .getSourceVertexName()));
-    Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
+        .getSourceVertexName()));
+    assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(0)
         .getInputDescriptor().getClassName())
         || "i3_v2".equals(v3.getInputSpecList(0).get(0)
-            .getInputDescriptor().getClassName()));
-    Assert.assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
+        .getInputDescriptor().getClassName()));
+    assertTrue("i3_v1".equals(v3.getInputSpecList(0).get(1)
         .getInputDescriptor().getClassName())
         || "i3_v2".equals(v3.getInputSpecList(0).get(1)
-            .getInputDescriptor().getClassName()));
+        .getInputDescriptor().getClassName()));
 
-    Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
+    assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(0)
         .getDestinationVertexName())
         || "vertex5".equals(v3.getOutputSpecList(0).get(0)
-            .getDestinationVertexName()));
-    Assert.assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
+        .getDestinationVertexName()));
+    assertTrue("vertex4".equals(v3.getOutputSpecList(0).get(1)
         .getDestinationVertexName())
         || "vertex5".equals(v3.getOutputSpecList(0).get(1)
-            .getDestinationVertexName()));
-    Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
+        .getDestinationVertexName()));
+    assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(0)
         .getOutputDescriptor().getClassName())
         || "o3_v5".equals(v3.getOutputSpecList(0).get(0)
-            .getOutputDescriptor().getClassName()));
-    Assert.assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
+        .getOutputDescriptor().getClassName()));
+    assertTrue("o3_v4".equals(v3.getOutputSpecList(0).get(1)
         .getOutputDescriptor().getClassName())
         || "o3_v5".equals(v3.getOutputSpecList(0).get(1)
-            .getOutputDescriptor().getClassName()));
+        .getOutputDescriptor().getClassName()));
   }
 
   @Test(timeout = 5000)
@@ -1940,13 +1941,13 @@ public class TestVertexImpl {
     Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
         Collections.singletonMap(
        v1.getName(), mockEdgeManagerDescriptor);
-    Assert.assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
-    Assert.assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
+    assertTrue(v3.setParallelism(1, null, edgeManagerDescriptors, null));
+    assertTrue(v3.sourceVertices.get(v1).getEdgeManager() instanceof
         EdgeManagerForTest);
     Assert.assertEquals(1, v3.getTotalTasks());
     Assert.assertEquals(1, tasks.size());
     // the last one is removed
-    Assert.assertTrue(tasks.keySet().iterator().next().equals(firstTask));
+    assertTrue(tasks.keySet().iterator().next().equals(firstTask));
 
   }
   
@@ -1993,7 +1994,7 @@ public class TestVertexImpl {
     Edge edge = edges.get("e4");
     EdgeManagerPlugin em = edge.getEdgeManager();
     EdgeManagerForTest originalEm = (EdgeManagerForTest) em;
-    Assert.assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
+    assertTrue(Arrays.equals(edgePayload, originalEm.getEdgeManagerContext()
         .getUserPayload().deepCopyAsArray()));
 
     UserPayload userPayload = UserPayload.create(ByteBuffer.wrap(new String("foo").getBytes()));
@@ -2007,7 +2008,7 @@ public class TestVertexImpl {
 
     Map<String, EdgeManagerPluginDescriptor> edgeManagerDescriptors =
         Collections.singletonMap(v3.getName(), edgeManagerDescriptor);
-    Assert.assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
+    assertTrue(v5.setParallelism(v5.getTotalTasks() - 1, null,
         edgeManagerDescriptors, null)); // Must decrease.
 
     VertexImpl v5Impl = (VertexImpl) v5;
@@ -2015,10 +2016,10 @@ public class TestVertexImpl {
     EdgeManagerPlugin modifiedEdgeManager = v5Impl.sourceVertices.get(v3)
         .getEdgeManager();
     Assert.assertNotNull(modifiedEdgeManager);
-    Assert.assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
+    assertTrue(modifiedEdgeManager instanceof EdgeManagerForTest);
 
     // Ensure initialize() is called with the correct payload
-    Assert.assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
+    assertTrue(Arrays.equals(userPayload.deepCopyAsArray(),
         ((EdgeManagerForTest) modifiedEdgeManager).getUserPayload().deepCopyAsArray()));
   }
 
@@ -2092,7 +2093,7 @@ public class TestVertexImpl {
     Assert.assertEquals(VertexTerminationCause.OWN_TASK_FAILURE, v.getTerminationCause());
     String diagnostics =
         StringUtils.join(v.getDiagnostics(), ",").toLowerCase();
-    Assert.assertTrue(diagnostics.contains("task failed"
+    assertTrue(diagnostics.contains("task failed"
         + ", taskid=" + t1.toString()));
   }
 
@@ -2104,7 +2105,7 @@ public class TestVertexImpl {
     String diagnostics =
         StringUtils.join(v2.getDiagnostics(), ",").toLowerCase();
     LOG.info("diagnostics v2: " + diagnostics);
-    Assert.assertTrue(diagnostics.contains(
+    assertTrue(diagnostics.contains(
         "vertex received kill in inited state"));
   }
 
@@ -2118,7 +2119,7 @@ public class TestVertexImpl {
     killVertex(v3);
     String diagnostics =
         StringUtils.join(v3.getDiagnostics(), ",").toLowerCase();
-    Assert.assertTrue(diagnostics.contains(
+    assertTrue(diagnostics.contains(
         "vertex received kill while in running state"));
   }
 
@@ -2200,7 +2201,7 @@ public class TestVertexImpl {
     Assert.assertNull(v2.getOutputCommitter("output"));
 
     VertexImpl v6 = vertices.get("vertex6");
-    Assert.assertTrue(v6.getOutputCommitter("outputx")
+    assertTrue(v6.getOutputCommitter("outputx")
         instanceof CountingOutputCommitter);
   }
 
@@ -2208,11 +2209,11 @@ public class TestVertexImpl {
   public void testVertexManagerInit() {
     initAllVertices(VertexState.INITED);
     VertexImpl v2 = vertices.get("vertex2");
-    Assert.assertTrue(v2.getVertexManager().getPlugin()
+    assertTrue(v2.getVertexManager().getPlugin()
         instanceof ImmediateStartVertexManager);
 
     VertexImpl v6 = vertices.get("vertex6");
-    Assert.assertTrue(v6.getVertexManager().getPlugin()
+    assertTrue(v6.getVertexManager().getPlugin()
         instanceof ShuffleVertexManager);
   }
 
@@ -3020,6 +3021,306 @@ public class TestVertexImpl {
     }
   }
 
+  @Test(timeout = 5000)
+  /**
+   * Ref: TEZ-1494
+   * If broadcast, one-to-one or custom edges are present in source, tasks should not start until
+   * 1 task from each source vertex is complete.
+   */
+  public void testTaskSchedulingWithCustomEdges() {
+    setupPreDagCreation();
+    dagPlan = createCustomDAGWithCustomEdges();
+    setupPostDagCreation();
+
+    /**
+     *
+     *   M2 --(SG)--> R3 --(B)--\
+     *                           \
+     *   M7 --(B)---------------->M5 ---(SG)--> R6
+     *                            /
+     *   M8---(C)--------------->/
+     */
+
+    //init M2
+    VertexImpl m2 = vertices.get("M2");
+    VertexImpl m7 = vertices.get("M7");
+    VertexImpl r3 = vertices.get("R3");
+    VertexImpl m5 = vertices.get("M5");
+    VertexImpl m8 = vertices.get("M8");
+
+    initVertex(m2);
+    initVertex(m7);
+    initVertex(m8);
+    assertTrue(m7.getState().equals(VertexState.INITED));
+    assertTrue(m5.getState().equals(VertexState.INITED));
+    assertTrue(m8.getState().equals(VertexState.INITED));
+    assertTrue(m7.getVertexManager().getPlugin() instanceof ImmediateStartVertexManager);
+
+    //Start M2; Let tasks complete in M2; Also let 1 task complete in R3
+    dispatcher.getEventHandler().handle(new VertexEvent(m2.getVertexId(), VertexEventType.V_START));
+    dispatcher.await();
+    VertexEventTaskAttemptCompleted taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m2.getVertexId(),0), 0), TaskAttemptStateInternal.SUCCEEDED);
+    VertexEventTaskCompleted taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m2
+        .getVertexId(), 0), TaskState.SUCCEEDED);
+    dispatcher.getEventHandler().handle(taskAttemptCompleted);
+    dispatcher.getEventHandler().handle(taskCompleted);
+    dispatcher.await();
+    taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(r3.getVertexId(),0), 0),
+            TaskAttemptStateInternal.SUCCEEDED);
+    taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(r3
+        .getVertexId(), 0), TaskState.SUCCEEDED);
+    dispatcher.getEventHandler().handle(taskAttemptCompleted);
+    dispatcher.getEventHandler().handle(taskCompleted);
+    dispatcher.await();
+    assertTrue(m2.getState().equals(VertexState.SUCCEEDED));
+    assertTrue(m5.numSuccessSourceAttemptCompletions == 1);
+    assertTrue(m5.getState().equals(VertexState.INITED));
+
+    //R3 should be in running state as it has one task completed, and rest are pending
+    assertTrue(r3.getState().equals(VertexState.RUNNING));
+
+    //Let us start M7; M5 should start not start as it is dependent on M8 as well
+    dispatcher.getEventHandler().handle(new VertexEvent(m7.getVertexId(),VertexEventType.V_START));
+    dispatcher.await();
+    //Let one of the tasks get over in M7 as well.
+    taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m7.getVertexId(),0), 0),
+            TaskAttemptStateInternal.SUCCEEDED);
+    taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m7
+        .getVertexId(), 0), TaskState.SUCCEEDED);
+    dispatcher.getEventHandler().handle(taskAttemptCompleted);
+    dispatcher.getEventHandler().handle(taskCompleted);
+    dispatcher.await();
+    assertTrue(m5.numSuccessSourceAttemptCompletions == 2);
+
+    //M5 should be in INITED state, as it depends on M8
+    assertTrue(m5.getState().equals(VertexState.INITED));
+    for(Task task : m5.getTasks().values()) {
+      assertTrue(task.getState().equals(TaskState.NEW));
+    }
+
+    //Let us start M8; M5 should start now
+    dispatcher.getEventHandler().handle(new VertexEvent(m8.getVertexId(),VertexEventType.V_START));
+    dispatcher.await();
+
+    //M5 in running state. But tasks should not be scheduled until M8 finishes a task.
+    assertTrue(m5.getState().equals(VertexState.RUNNING));
+    for(Task task : m5.getTasks().values()) {
+      assertTrue(task.getState().equals(TaskState.NEW));
+    }
+
+    //Let one of the tasks get over in M8 as well. This should trigger tasks to be scheduled in M5
+    taskAttemptCompleted = new VertexEventTaskAttemptCompleted
+        (TezTaskAttemptID.getInstance(TezTaskID.getInstance(m8.getVertexId(),0), 0),
+            TaskAttemptStateInternal.SUCCEEDED);
+    taskCompleted = new VertexEventTaskCompleted(TezTaskID.getInstance(m8
+        .getVertexId(), 0), TaskState.SUCCEEDED);
+    dispatcher.getEventHandler().handle(taskAttemptCompleted);
+    dispatcher.getEventHandler().handle(taskCompleted);
+    dispatcher.await();
+
+    assertTrue(m5.numSuccessSourceAttemptCompletions == 3);
+    //Ensure all tasks in M5 are in scheduled state
+    for(Task task : m5.getTasks().values()) {
+      assertTrue(task.getState().equals(TaskState.SCHEDULED));
+    }
+  }
+
+  //For TEZ-1494
+  private DAGPlan createCustomDAGWithCustomEdges() {
+    /**
+     *
+     *   M2 --(SG)--> R3 --(B)--\
+     *                           \
+     *   M7 --(B)---------------->M5 ---(SG)--> R6
+     *                            /
+     *   M8---(C)--------------->/
+     */
+    DAGPlan dag = DAGPlan.newBuilder().setName("TestSamplerDAG")
+        .addVertex(VertexPlan.newBuilder()
+                .setName("M2")
+                .setProcessorDescriptor(
+                    TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
+                .setType(PlanVertexType.NORMAL)
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
+                .setTaskConfig(PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(1)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("M2.class")
+                        .build()
+                )
+                .addOutEdgeId("M2_R3")
+                .build()
+        )
+        .addVertex(VertexPlan.newBuilder()
+                .setName("M8")
+                .setProcessorDescriptor(
+                    TezEntityDescriptorProto.newBuilder().setClassName("M8.class"))
+                .setType(PlanVertexType.NORMAL)
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host1").addRack("rack1").build())
+                .setTaskConfig(PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(1)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("M8.class")
+                        .build()
+                )
+                .addOutEdgeId("M8_M5")
+                .build()
+        )
+         .addVertex(VertexPlan.newBuilder()
+                 .setName("R3")
+                 .setProcessorDescriptor(
+                     TezEntityDescriptorProto.newBuilder().setClassName("M2.class"))
+                 .setType(PlanVertexType.NORMAL)
+                 .addTaskLocationHint(
+                     PlanTaskLocationHint.newBuilder().addHost("host2").addRack("rack1").build())
+                 .setTaskConfig(PlanTaskConfiguration.newBuilder()
+                         .setNumTasks(10)
+                         .setVirtualCores(4)
+                         .setMemoryMb(1024)
+                         .setJavaOpts("")
+                         .setTaskModule("R3.class")
+                         .build()
+                 )
+                 .addInEdgeId("M2_R3")
+                 .addOutEdgeId("R3_M5")
+                 .build()
+         )
+        .addVertex(VertexPlan.newBuilder()
+                .setName("M5")
+                .setProcessorDescriptor(
+                    TezEntityDescriptorProto.newBuilder().setClassName("M5.class"))
+                .setType(PlanVertexType.NORMAL)
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
+                .setTaskConfig(PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(10)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("M5.class")
+                        .build()
+                )
+                .addInEdgeId("R3_M5")
+                .addInEdgeId("M7_M5")
+                .addInEdgeId("M8_M5")
+                .addOutEdgeId("M5_R6")
+                .build()
+        )
+        .addVertex(VertexPlan.newBuilder()
+                .setName("M7")
+                .setProcessorDescriptor(
+                    TezEntityDescriptorProto.newBuilder().setClassName("M7.class"))
+                .setType(PlanVertexType.NORMAL)
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host4").addRack("rack1").build())
+                .setTaskConfig(PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(10)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("M7.class")
+                        .build()
+                )
+                .addOutEdgeId("M7_M5")
+                .build()
+        )
+        .addVertex(VertexPlan.newBuilder()
+                .setName("R6")
+                .setProcessorDescriptor(
+                    TezEntityDescriptorProto.newBuilder().setClassName("R6.class"))
+                .setType(PlanVertexType.NORMAL)
+                .addTaskLocationHint(
+                    PlanTaskLocationHint.newBuilder().addHost("host3").addRack("rack1").build())
+                .setTaskConfig(PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(1)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("R6.class")
+                        .build()
+                )
+                .addInEdgeId("M5_R6")
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3"))
+                .setInputVertexName("M2")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M2_R3.class"))
+                .setOutputVertexName("R3")
+                .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
+                .setId("M2_R3")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5"))
+                .setInputVertexName("R3")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("R3_M5.class"))
+                .setOutputVertexName("M5")
+                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+                .setId("R3_M5")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5"))
+                .setInputVertexName("M7")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M7_M5.class"))
+                .setOutputVertexName("M5")
+                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+                .setId("M7_M5")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6"))
+                .setInputVertexName("M5")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M5_R6.class"))
+                .setOutputVertexName("R6")
+                .setDataMovementType(PlanEdgeDataMovementType.SCATTER_GATHER)
+                .setId("M5_R6")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5"))
+                .setInputVertexName("M8")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("M8_M5.class"))
+                .setEdgeManager(
+                    TezEntityDescriptorProto.newBuilder()
+                        .setClassName(EdgeManagerForTest.class.getName())
+                        .setUserPayload(ByteString.copyFrom(edgePayload))
+                        .build())
+                .setOutputVertexName("M5")
+                .setDataMovementType(PlanEdgeDataMovementType.CUSTOM)
+                .setId("M8_M5")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .build();
+
+    return dag;
+  }
+
   @SuppressWarnings("unchecked")
   @Test(timeout = 5000)
   public void testVertexWithInitializerSuccess() {
@@ -3215,6 +3516,7 @@ public class TestVertexImpl {
           dispatcher.getEventHandler(), taskAttemptListener,
           clock, thh, true, appContext, vertexLocationHint, null, taskSpecificLaunchCmdOption,
           updateTracker);
+      v.setInputVertices(new HashMap());
       vertexIdMap.put(vId, v);
       vertices.put(v.getName(), v);
       v.handle(new VertexEvent(vId, VertexEventType.V_INIT));
@@ -3454,8 +3756,8 @@ public class TestVertexImpl {
     List<GroupInputSpec> groupInSpec = vC.getGroupInputSpecList(0);
     Assert.assertEquals(1, groupInSpec.size());
     Assert.assertEquals("Group", groupInSpec.get(0).getGroupName());
-    Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
-    Assert.assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
+    assertTrue(groupInSpec.get(0).getGroupVertices().contains("A"));
+    assertTrue(groupInSpec.get(0).getGroupVertices().contains("B"));
     groupInSpec.get(0).getMergedInputDescriptor().getClassName().equals("Group.class");
   }
   


[12/25] git commit: TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts. (hitesh)

Posted by je...@apache.org.
TEZ-1575. MRRSleepJob does not pick MR settings for container size and java opts. (hitesh)


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

Branch: refs/heads/TEZ-8
Commit: b62298e1054c934e4ee8909b7ba3165ce344ea8b
Parents: d1ea44d
Author: Hitesh Shah <hi...@apache.org>
Authored: Fri Sep 12 11:02:51 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Fri Sep 12 11:02:51 2014 -0700

----------------------------------------------------------------------
 .../tez/mapreduce/examples/MRRSleepJob.java     | 25 +++++++++++++-------
 1 file changed, 16 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/b62298e1/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
index 3f9d222..de8a1ad 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/MRRSleepJob.java
@@ -525,9 +525,11 @@ public class MRRSleepJob extends Configured implements Tool {
     int numTasks = generateSplitsInAM ? -1 : numMapper;
 
     Vertex mapVertex = Vertex.create("map", ProcessorDescriptor.create(
-        MapProcessor.class.getName()).setUserPayload(mapUserPayload), numTasks)
-        .addTaskLocalFiles(commonLocalResources);
-    mapVertex.addDataSource("MRInput", dataSource);
+        MapProcessor.class.getName()).setUserPayload(mapUserPayload), numTasks,
+        MRHelpers.getResourceForMRMapper(mapStageConf));
+    mapVertex.addTaskLocalFiles(commonLocalResources)
+        .addDataSource("MRInput", dataSource)
+        .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRMapper(mapStageConf));
     vertices.add(mapVertex);
 
     if (iReduceStagesCount > 0
@@ -538,8 +540,11 @@ public class MRRSleepJob extends Configured implements Tool {
         UserPayload iReduceUserPayload = TezUtils.createUserPayloadFromConf(iconf);
         Vertex ivertex = Vertex.create("ireduce" + (i + 1),
             ProcessorDescriptor.create(ReduceProcessor.class.getName()).
-                setUserPayload(iReduceUserPayload), numIReducer);
-        ivertex.addTaskLocalFiles(commonLocalResources);
+                setUserPayload(iReduceUserPayload), numIReducer,
+            MRHelpers.getResourceForMRReducer(intermediateReduceStageConfs[i]));
+        ivertex.addTaskLocalFiles(commonLocalResources)
+            .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(
+                intermediateReduceStageConfs[i]));
         vertices.add(ivertex);
       }
     }
@@ -548,10 +553,12 @@ public class MRRSleepJob extends Configured implements Tool {
     if (numReducer > 0) {
       UserPayload reducePayload = TezUtils.createUserPayloadFromConf(finalReduceConf);
       finalReduceVertex = Vertex.create("reduce", ProcessorDescriptor.create(
-          ReduceProcessor.class.getName()).setUserPayload(reducePayload), numReducer);
-      finalReduceVertex.addTaskLocalFiles(commonLocalResources);
-      finalReduceVertex.addDataSink("MROutput", MROutputLegacy.createConfigBuilder(finalReduceConf,
-          NullOutputFormat.class).build());
+          ReduceProcessor.class.getName()).setUserPayload(reducePayload), numReducer,
+          MRHelpers.getResourceForMRReducer(finalReduceConf));
+      finalReduceVertex.addTaskLocalFiles(commonLocalResources)
+          .addDataSink("MROutput", MROutputLegacy.createConfigBuilder(
+              finalReduceConf, NullOutputFormat.class).build())
+          .setTaskLaunchCmdOpts(MRHelpers.getJavaOptsForMRReducer(finalReduceConf));
       vertices.add(finalReduceVertex);
     } else {
       // Map only job


[02/25] git commit: TEZ-850. Recovery unit tests. (Jeff Zhang via hitesh)

Posted by je...@apache.org.
TEZ-850. Recovery unit tests. (Jeff Zhang via hitesh)


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

Branch: refs/heads/TEZ-8
Commit: f65e65aea8cbdb44dd65c6590fbe38dd84413a5a
Parents: d6589d3
Author: Hitesh Shah <hi...@apache.org>
Authored: Thu Sep 11 13:22:21 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Thu Sep 11 13:22:21 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |   1 +
 .../apache/tez/dag/app/dag/impl/DAGImpl.java    |  45 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   |   3 +-
 .../apache/tez/dag/app/dag/impl/TaskImpl.java   |  12 +-
 .../apache/tez/dag/app/dag/impl/VertexImpl.java | 240 +++---
 .../tez/dag/app/dag/impl/TestDAGImpl.java       |   6 +-
 .../tez/dag/app/dag/impl/TestDAGRecovery.java   | 514 +++++++++++
 .../app/dag/impl/TestTaskAttemptRecovery.java   | 178 ++++
 .../tez/dag/app/dag/impl/TestTaskRecovery.java  | 629 +++++++++++---
 .../dag/app/dag/impl/TestVertexRecovery.java    | 860 +++++++++++++++++++
 10 files changed, 2243 insertions(+), 245 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 032438f..4fc7e83 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -9,6 +9,7 @@ INCOMPATIBLE CHANGES
 ALL CHANGES:
   TEZ-1544. Link to release artifacts for 0.5.0 does not point to a specific link for 0.5.0.
   TEZ-1559. Add system tests for AM recovery.
+  TEZ-850. Recovery unit tests.
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 24d2e6b..daaa81b 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
@@ -139,7 +139,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   private final TaskAttemptListener taskAttemptListener;
   private final TaskHeartbeatHandler taskHeartbeatHandler;
   private final Object tasksSyncHandle = new Object();
-  
+
   private volatile boolean committedOrAborted = false;
   private volatile boolean allOutputsCommitted = false;
   boolean commitAllOutputsOnSuccess = true;
@@ -157,7 +157,8 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   private final StateChangeNotifier entityUpdateTracker;
 
   volatile Map<TezVertexID, Vertex> vertices = new HashMap<TezVertexID, Vertex>();
-  private Map<String, Edge> edges = new HashMap<String, Edge>();
+  @VisibleForTesting
+  Map<String, Edge> edges = new HashMap<String, Edge>();
   private TezCounters dagCounters = new TezCounters();
   private Object fullCountersLock = new Object();
   private TezCounters fullCounters = null;
@@ -359,14 +360,18 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
   private DAGTerminationCause terminationCause;
   private Credentials credentials;
 
-  private long initTime;
-  private long startTime;
-  private long finishTime;
-  
+  @VisibleForTesting
+  long initTime;
+  @VisibleForTesting
+  long startTime;
+  @VisibleForTesting
+  long finishTime;
+
   Map<String, VertexGroupInfo> vertexGroups = Maps.newHashMap();
   Map<String, List<VertexGroupInfo>> vertexGroupInfo = Maps.newHashMap();
   private DAGState recoveredState = DAGState.NEW;
-  private boolean recoveryCommitInProgress = false;
+  @VisibleForTesting
+  boolean recoveryCommitInProgress = false;
   Map<String, Boolean> recoveredGroupCommits = new HashMap<String, Boolean>();
 
   static class VertexGroupInfo {
@@ -381,7 +386,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
       groupMembers = Sets.newHashSet(groupInfo.getGroupMembersList());
       edgeMergedInputs = Maps.newHashMapWithExpectedSize(groupInfo.getEdgeMergedInputsCount());
       for (PlanGroupInputEdgeInfo edgInfo : groupInfo.getEdgeMergedInputsList()) {
-        edgeMergedInputs.put(edgInfo.getDestVertexName(), 
+        edgeMergedInputs.put(edgInfo.getDestVertexName(),
             DagTypeConverters.convertInputDescriptorFromDAGPlan(edgInfo.getMergedInput()));
       }
       outputs = Sets.newHashSet(groupInfo.getOutputsList());
@@ -706,7 +711,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
       }
     }
   }
-  
+
   private boolean commitOutput(String outputName, OutputCommitter outputCommitter) {
     final OutputCommitter committer = outputCommitter;
     try {
@@ -723,7 +728,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     }
     return false;
   }
-  
+
   private synchronized boolean commitOrAbortOutputs(boolean dagSucceeded) {
     if (this.committedOrAborted) {
       LOG.info("Ignoring multiple output commit/abort");
@@ -731,7 +736,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     }
     LOG.info("Calling DAG commit/abort for dag: " + getID());
     this.committedOrAborted = true;
-    
+
     boolean successfulOutputsAlreadyCommitted = !commitAllOutputsOnSuccess;
     boolean failedWhileCommitting = false;
     if (dagSucceeded && !successfulOutputsAlreadyCommitted) {
@@ -772,7 +777,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
           LOG.info("No output committers for vertex: " + vertex.getName());
           continue;
         }
-        Map<String, OutputCommitter> outputCommitters = 
+        Map<String, OutputCommitter> outputCommitters =
             new HashMap<String, OutputCommitter>(vertex.getOutputCommitters());
         Set<String> sharedOutputs = vertex.getSharedOutputs();
         // remove shared outputs
@@ -793,7 +798,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
           LOG.info("Committing output: " + entry.getKey() + " for vertex: "
               + vertex.getVertexId());
           if (vertex.getState() != VertexState.SUCCEEDED) {
-            throw new TezUncheckedException("Vertex: " + vertex.getName() + 
+            throw new TezUncheckedException("Vertex: " + vertex.getName() +
                 " not in SUCCEEDED state. State= " + vertex.getState());
           }
           if (!commitOutput(entry.getKey(), entry.getValue())) {
@@ -803,11 +808,11 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
         }
       }
     }
-    
+
     if (failedWhileCommitting) {
       LOG.info("DAG: " + getID() + " failed while committing");
     }
-        
+
     if (!dagSucceeded || failedWhileCommitting) {
       // come here because dag failed or
       // dag succeeded and all or none semantics were on and a commit failed
@@ -1026,9 +1031,9 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     if (finishTime == 0) {
       setFinishTime();
     }
-    
+
     boolean allOutputsCommitted = commitOrAbortOutputs(finalState == DAGState.SUCCEEDED);
-    
+
     if (finalState == DAGState.SUCCEEDED && !allOutputsCommitted) {
       finalState = DAGState.FAILED;
       trySetTerminationCause(DAGTerminationCause.COMMIT_FAILURE);
@@ -1057,7 +1062,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     LOG.info("DAG: " + getID() + " finished with state: " + finalState);
     return finalState;
   }
-  
+
   private DAGStatus.State getDAGStatusFromState(DAGState finalState) {
     switch (finalState) {
       case NEW:
@@ -1631,7 +1636,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
           + ", numFailedVertices=" + job.numFailedVertices
           + ", numKilledVertices=" + job.numKilledVertices
           + ", numVertices=" + job.numVertices);
-      
+
       if (failed) {
         return DAGState.TERMINATING;
       }
@@ -1724,7 +1729,7 @@ public class DAGImpl implements org.apache.tez.dag.app.dag.DAG,
     addDiagnostic("Vertex re-running"
       + ", vertexName=" + vertex.getName()
       + ", vertexId=" + vertex.getVertexId());
-    
+
     if (!commitAllOutputsOnSuccess) {
       // partial output may already have been committed. fail if so
       List<VertexGroupInfo> groupList = vertexGroupInfo.get(vertex.getName());

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 e63dbf5..7ba90b5 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
@@ -137,7 +137,8 @@ public class TaskAttemptImpl implements TaskAttempt,
   private String nodeHttpAddress;
   private String nodeRackName;
 
-  private TaskAttemptStatus reportedStatus;
+  @VisibleForTesting
+  TaskAttemptStatus reportedStatus;
   private DAGCounter localityCounter;
 
   // Used to store locality information when

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 ecd2bcc..1dd711b 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
@@ -116,7 +116,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   protected final AppContext appContext;
   private final Resource taskResource;
   private final ContainerContext containerContext;
-  private long scheduledTime;
+  @VisibleForTesting
+  long scheduledTime;
 
   private final List<TezEvent> tezEventsForTaskAttempts = new ArrayList<TezEvent>();
   private static final List<TezEvent> EMPTY_TASK_ATTEMPT_TEZ_EVENTS =
@@ -124,7 +125,8 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
 
   // counts the number of attempts that are either running or in a state where
   //  they will come to be running when they get a Container
-  private int numberUncompletedAttempts = 0;
+  @VisibleForTesting
+  int numberUncompletedAttempts = 0;
 
   private boolean historyTaskStartGenerated = false;
 
@@ -290,11 +292,13 @@ public class TaskImpl implements Task, EventHandler<TaskEvent> {
   //saying COMMIT_PENDING
   private TezTaskAttemptID commitAttempt;
 
-  private TezTaskAttemptID successfulAttempt;
+  @VisibleForTesting
+  TezTaskAttemptID successfulAttempt;
 
   @VisibleForTesting
   int failedAttempts;
-  private int finishedAttempts;//finish are total of success, failed and killed
+  @VisibleForTesting
+  int finishedAttempts;//finish are total of success, failed and killed
 
   private final boolean leafVertex;
   private TaskState recoveredState = TaskState.NEW;

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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 ff556ba..31240cb 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
@@ -205,9 +205,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
   //fields initialized in init
 
-  private int numStartedSourceVertices = 0;
-  private int numInitedSourceVertices = 0;
-  private int numRecoveredSourceVertices = 0;
+  @VisibleForTesting
+  int numStartedSourceVertices = 0;
+  @VisibleForTesting
+  int numInitedSourceVertices = 0;
+  @VisibleForTesting
+  int numRecoveredSourceVertices = 0;
 
   private int distanceFromRoot = 0;
 
@@ -238,7 +241,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       new VertexStateChangedCallback();
 
   private VertexState recoveredState = VertexState.NEW;
-  private List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
+  @VisibleForTesting
+  List<TezEvent> recoveredEvents = new ArrayList<TezEvent>();
   private boolean vertexAlreadyInitialized = false;
 
   protected static final
@@ -254,7 +258,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
                       VertexState.INITIALIZING, VertexState.FAILED),
                   VertexEventType.V_INIT,
                   new InitTransition())
-          .addTransition(VertexState.NEW, 
+          .addTransition(VertexState.NEW,
                 EnumSet.of(VertexState.NEW),
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
                   new NullEdgeInitializedTransition())
@@ -308,7 +312,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
           // Transitions from INITIALIZING state
           .addTransition(VertexState.INITIALIZING,
-              EnumSet.of(VertexState.INITIALIZING, VertexState.INITED, 
+              EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
                   VertexState.FAILED),
               VertexEventType.V_ROOT_INPUT_INITIALIZED,
               new RootInputInitializedTransition())
@@ -341,14 +345,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           .addTransition(VertexState.INITIALIZING, VertexState.ERROR,
               VertexEventType.V_INTERNAL_ERROR,
               INTERNAL_ERROR_TRANSITION)
-          .addTransition(VertexState.INITIALIZING, 
+          .addTransition(VertexState.INITIALIZING,
               EnumSet.of(VertexState.INITIALIZING, VertexState.INITED,
                   VertexState.FAILED),
                   VertexEventType.V_NULL_EDGE_INITIALIZED,
                   new NullEdgeInitializedTransition())
 
           // Transitions from INITED state
-          // SOURCE_VERTEX_STARTED - for sources which determine parallelism, 
+          // SOURCE_VERTEX_STARTED - for sources which determine parallelism,
           // they must complete before this vertex can start.
           .addTransition
               (VertexState.INITED,
@@ -358,14 +362,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           .addTransition(VertexState.INITED, VertexState.INITED,
               VertexEventType.V_SOURCE_VERTEX_STARTED,
               new SourceVertexStartedTransition())
-          .addTransition(VertexState.INITED, 
+          .addTransition(VertexState.INITED,
               EnumSet.of(VertexState.INITED),
               VertexEventType.V_ONE_TO_ONE_SOURCE_SPLIT,
               new OneToOneSourceSplitTransition())
           .addTransition(VertexState.INITED,  VertexState.INITED,
               VertexEventType.V_SOURCE_TASK_ATTEMPT_COMPLETED,
               SOURCE_TASK_ATTEMPT_COMPLETED_EVENT_TRANSITION)
-          .addTransition(VertexState.INITED, 
+          .addTransition(VertexState.INITED,
               EnumSet.of(VertexState.RUNNING, VertexState.INITED),
               VertexEventType.V_START,
               new StartTransition())
@@ -393,7 +397,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
                   VertexState.ERROR),
               VertexEventType.V_TASK_COMPLETED,
               new TaskCompletedTransition())
-          .addTransition(VertexState.RUNNING, 
+          .addTransition(VertexState.RUNNING,
               EnumSet.of(VertexState.RUNNING),
               VertexEventType.V_ONE_TO_ONE_SOURCE_SPLIT,
               new OneToOneSourceSplitTransition())
@@ -455,7 +459,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               VertexEventType.V_ROUTE_EVENT,
               ROUTE_EVENT_TRANSITION)
           .addTransition(
-              VertexState.SUCCEEDED, 
+              VertexState.SUCCEEDED,
               EnumSet.of(VertexState.FAILED, VertexState.ERROR),
               VertexEventType.V_TASK_COMPLETED,
               new TaskCompletedAfterVertexSuccessTransition())
@@ -551,17 +555,27 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   private final StateMachineTez<VertexState, VertexEventType, VertexEvent, VertexImpl> stateMachine;
 
   //changing fields while the vertex is running
-  private int numTasks;
-  private int completedTaskCount = 0;
-  private int succeededTaskCount = 0;
-  private int failedTaskCount = 0;
-  private int killedTaskCount = 0;
-
-  private long initTimeRequested; // Time at which INIT request was received.
-  private long initedTime; // Time when entering state INITED
-  private long startTimeRequested; // Time at which START request was received.
-  private long startedTime; // Time when entering state STARTED
-  private long finishTime;
+  @VisibleForTesting
+  int numTasks;
+  @VisibleForTesting
+  int completedTaskCount = 0;
+  @VisibleForTesting
+  int succeededTaskCount = 0;
+  @VisibleForTesting
+  int failedTaskCount = 0;
+  @VisibleForTesting
+  int killedTaskCount = 0;
+
+  @VisibleForTesting
+  long initTimeRequested; // Time at which INIT request was received.
+  @VisibleForTesting
+  long initedTime; // Time when entering state INITED
+  @VisibleForTesting
+  long startTimeRequested; // Time at which START request was received.
+  @VisibleForTesting
+  long startedTime; // Time when entering state STARTED
+  @VisibleForTesting
+  long finishTime;
   private float progress;
 
   private final TezVertexID vertexId;  //runtime assigned id.
@@ -576,14 +590,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   private Map<Vertex, Edge> targetVertices;
   Set<Edge> uninitializedEdges = Sets.newHashSet();
 
-  private Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+  private Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
     rootInputDescriptors;
-  private Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> 
+  private Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
     additionalOutputs;
   private Map<String, OutputCommitter> outputCommitters;
   private Map<String, InputSpecUpdate> rootInputSpecs = new HashMap<String, InputSpecUpdate>();
   private static final InputSpecUpdate DEFAULT_ROOT_INPUT_SPECS = InputSpecUpdate
-      .getDefaultSinglePhysicalInputSpecUpdate(); 
+      .getDefaultSinglePhysicalInputSpecUpdate();
   private final List<OutputSpec> additionalOutputSpecs = new ArrayList<OutputSpec>();
   private Set<String> inputsWithInitializers;
   private int numInitializedInputs;
@@ -598,7 +612,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   private RootInputInitializerManager rootInputInitializerManager;
 
   VertexManager vertexManager;
-  
+
   private final UserGroupInformation dagUgi;
 
   private boolean parallelismSet = false;
@@ -607,20 +621,22 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   private AtomicBoolean committed = new AtomicBoolean(false);
   private AtomicBoolean aborted = new AtomicBoolean(false);
   private boolean commitVertexOutputs = false;
-  
+
   private Map<String, VertexGroupInfo> dagVertexGroups;
-  
+
   private TaskLocationHint taskLocationHints[];
   private Map<String, LocalResource> localResources;
   private Map<String, String> environment;
   private final String javaOpts;
   private final ContainerContext containerContext;
   private VertexTerminationCause terminationCause;
-  
+
   private String logIdentifier;
-  private boolean recoveryCommitInProgress = false;
+  @VisibleForTesting
+  boolean recoveryCommitInProgress = false;
   private boolean summaryCompleteSeen = false;
-  private boolean hasCommitter = false;
+  @VisibleForTesting
+  boolean hasCommitter = false;
   private boolean vertexCompleteSeen = false;
   private Map<String,EdgeManagerPluginDescriptor> recoveredSourceEdgeManagers = null;
   private Map<String, InputSpecUpdate> recoveredRootInputSpecUpdates = null;
@@ -658,7 +674,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       logLocationHints(this.vertexName, vertexLocationHint);
     }
     setTaskLocationHints(vertexLocationHint);
-    
+
     this.dagUgi = appContext.getCurrentDAG().getDagUGI();
 
     this.taskResource = DagTypeConverters
@@ -897,12 +913,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       this.readLock.unlock();
     }
   }
-  
+
   @Override
   public TaskLocationHint getTaskLocationHint(TezTaskID taskId) {
     this.readLock.lock();
     try {
-      if (taskLocationHints == null || 
+      if (taskLocationHints == null ||
           taskLocationHints.length <= taskId.getId()) {
         return null;
       }
@@ -1081,8 +1097,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   }
 
   private void setTaskLocationHints(VertexLocationHint vertexLocationHint) {
-    if (vertexLocationHint != null && 
-        vertexLocationHint.getTaskLocationHints() != null && 
+    if (vertexLocationHint != null &&
+        vertexLocationHint.getTaskLocationHints() != null &&
         !vertexLocationHint.getTaskLocationHints().isEmpty()) {
       List<TaskLocationHint> locHints = vertexLocationHint.getTaskLocationHints();
       taskLocationHints = locHints.toArray(new TaskLocationHint[locHints.size()]);
@@ -1154,7 +1170,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
             }
           }
         }
-        
+
         // Restore any rootInputSpecUpdates which may have been registered during a parallelism
         // update.
         if (rootInputSpecUpdates != null) {
@@ -1166,7 +1182,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         writeLock.unlock();
       }
     }
-    Preconditions.checkArgument(parallelism >= 0, "Parallelism must be >=0. Value: " 
+    Preconditions.checkArgument(parallelism >= 0, "Parallelism must be >=0. Value: "
     + parallelism + " for vertex: " + logIdentifier);
     setVertexLocationHint(vertexLocationHint);
     writeLock.lock();
@@ -1175,7 +1191,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         LOG.info("Parallelism can only be set dynamically once per vertex: " + logIdentifier);
         return false;
       }
-      
+
       parallelismSet = true;
 
       // Input initializer/Vertex Manager/1-1 split expected to set parallelism.
@@ -1185,7 +1201,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               "Vertex state is not Initializing. Value: " + getState()
                   + " for vertex: " + logIdentifier);
         }
-        
+
         if(sourceEdgeManagers != null) {
           for(Map.Entry<String, EdgeManagerPluginDescriptor> entry : sourceEdgeManagers.entrySet()) {
             LOG.info("Replacing edge manager for source:"
@@ -1223,7 +1239,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         stateChangeNotifier.stateChanged(vertexId,
             new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
         this.createTasks();
-        LOG.info("Vertex " + getVertexId() + 
+        LOG.info("Vertex " + getVertexId() +
             " parallelism set to " + parallelism);
         if (canInitVertex()) {
           getEventHandler().handle(new VertexEvent(getVertexId(), VertexEventType.V_READY_TO_INIT));
@@ -1244,7 +1260,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           return false;
         }
         if (parallelism == numTasks) {
-          LOG.info("setParallelism same as current value: " + parallelism + 
+          LOG.info("setParallelism same as current value: " + parallelism +
               " for vertex: " + logIdentifier);
           Preconditions.checkArgument(sourceEdgeManagers != null,
               "Source edge managers or RootInputSpecs must be set when not changing parallelism");
@@ -1258,7 +1274,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         for (Edge edge : sourceVertices.values()) {
           edge.startEventBuffering();
         }
-  
+
         // assign to local variable of LinkedHashMap to make sure that changing
         // type of task causes compile error. We depend on LinkedHashMap for order
         LinkedHashMap<TezTaskID, Task> currentTasks = this.tasks;
@@ -1281,14 +1297,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           LOG.info("Removing task: " + entry.getKey());
           iter.remove();
         }
-        LOG.info("Vertex " + logIdentifier + 
+        LOG.info("Vertex " + logIdentifier +
             " parallelism set to " + parallelism + " from " + numTasks);
         int oldNumTasks = numTasks;
         this.numTasks = parallelism;
         stateChangeNotifier.stateChanged(vertexId,
             new VertexStateUpdateParallelismUpdated(vertexName, numTasks, oldNumTasks));
         assert tasks.size() == numTasks;
-  
+
         // set new edge managers
         if(sourceEdgeManagers != null) {
           for(Map.Entry<String, EdgeManagerPluginDescriptor> entry : sourceEdgeManagers.entrySet()) {
@@ -1320,17 +1336,17 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           edge.stopEventBuffering();
         }
       }
-      
+
       for (Map.Entry<Vertex, Edge> entry : targetVertices.entrySet()) {
         Edge edge = entry.getValue();
-        if (edge.getEdgeProperty().getDataMovementType() 
+        if (edge.getEdgeProperty().getDataMovementType()
             == DataMovementType.ONE_TO_ONE) {
           // inform these target vertices that we have changed parallelism
-          VertexEventOneToOneSourceSplit event = 
+          VertexEventOneToOneSourceSplit event =
               new VertexEventOneToOneSourceSplit(entry.getKey().getVertexId(),
                   getVertexId(),
-                  ((originalOneToOneSplitSource!=null) ? 
-                      originalOneToOneSplitSource : getVertexId()), 
+                  ((originalOneToOneSplitSource!=null) ?
+                      originalOneToOneSplitSource : getVertexId()),
                   numTasks);
           getEventHandler().handle(event);
         }
@@ -1339,7 +1355,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     } finally {
       writeLock.unlock();
     }
-    
+
     return true;
   }
 
@@ -1733,7 +1749,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     // no code, for now
   }
 
-  private ContainerContext getContainerContext(int taskIdx) {
+  @VisibleForTesting
+  ContainerContext getContainerContext(int taskIdx) {
     if (taskSpecificLaunchCmdOpts.addTaskSpecificLaunchCmdOption(vertexName, taskIdx)) {
       String jvmOpts = taskSpecificLaunchCmdOpts.getTaskSpecificOption(javaOpts, vertexName, taskIdx);
       ContainerContext context = new ContainerContext(this.localResources,
@@ -1806,9 +1823,9 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       if (rootInputDescriptors != null) {
         LOG.info("Root Inputs exist for Vertex: " + getName() + " : "
             + rootInputDescriptors);
-        for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input 
+        for (RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor> input
             : rootInputDescriptors.values()) {
-          if (input.getControllerDescriptor() != null && 
+          if (input.getControllerDescriptor() != null &&
               input.getControllerDescriptor().getClassName() != null) {
             if (inputsWithInitializers == null) {
               inputsWithInitializers = Sets.newHashSet();
@@ -1875,7 +1892,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     checkTaskLimits();
     return VertexState.INITED;
   }
-  
+
   private void assignVertexManager() {
     boolean hasBipartite = false;
     boolean hasOneToOne = false;
@@ -1895,12 +1912,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           hasCustom = true;
           break;
         default:
-          throw new TezUncheckedException("Unknown data movement type: " + 
+          throw new TezUncheckedException("Unknown data movement type: " +
               edge.getEdgeProperty().getDataMovementType());
         }
       }
     }
-    
+
     boolean hasUserVertexManager = vertexPlan.hasVertexManagerPlugin();
 
     if (hasUserVertexManager) {
@@ -2008,6 +2025,8 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           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:
@@ -2086,6 +2105,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
             vertex.vertexManager.onVertexStarted(vertex.pendingReportedSrcCompletions);
             endState = VertexState.RUNNING;
           } else {
+            // why succeeded here
             endState = VertexState.SUCCEEDED;
             vertex.finished(endState);
           }
@@ -2249,7 +2269,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           (vertex.sourceVertices == null || vertex.sourceVertices.containsKey(otherVertex) ||
           vertex.targetVertices == null || vertex.targetVertices.containsKey(otherVertex)),
           "Not connected to vertex " + otherVertex.getName() + " from vertex: " + vertex.logIdentifier);
-      LOG.info("Edge initialized for connection to vertex " + otherVertex.getName() + 
+      LOG.info("Edge initialized for connection to vertex " + otherVertex.getName() +
           " at vertex : " + vertex.logIdentifier);
       vertex.uninitializedEdges.remove(edge);
       if(vertex.getState() == VertexState.INITIALIZING && vertex.canInitVertex()) {
@@ -2596,7 +2616,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       if (state.equals(VertexState.FAILED)) {
         return state;
       }
-      // TODO move before to handle NEW state 
+      // TODO move before to handle NEW state
       if (vertex.targetVertices != null) {
         for (Edge e : vertex.targetVertices.values()) {
           if (e.getEdgeManager() == null) {
@@ -2619,7 +2639,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
           }
         }
       }
-      
+
       // Create tasks based on initial configuration, but don't start them yet.
       if (vertex.numTasks == -1) {
         LOG.info("Num tasks is -1. Expecting VertexManager/InputInitializers/1-1 split"
@@ -2633,7 +2653,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               vertex.appContext.getTaskScheduler().getNumClusterNodes(),
               vertex.getTaskResource(),
               vertex.appContext.getTaskScheduler().getTotalResources());
-          List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+          List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
               inputList = Lists.newArrayListWithCapacity(vertex.inputsWithInitializers.size());
           for (String inputName : vertex.inputsWithInitializers) {
             inputList.add(vertex.rootInputDescriptors.get(inputName));
@@ -2646,7 +2666,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         } else {
           boolean hasOneToOneUninitedSource = false;
           for (Map.Entry<Vertex, Edge> entry : vertex.sourceVertices.entrySet()) {
-            if (entry.getValue().getEdgeProperty().getDataMovementType() == 
+            if (entry.getValue().getEdgeProperty().getDataMovementType() ==
                 DataMovementType.ONE_TO_ONE) {
               if (entry.getKey().getTotalTasks() == -1) {
                 hasOneToOneUninitedSource = true;
@@ -2662,7 +2682,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
             LOG.info("Vertex will initialize via custom vertex manager. " + vertex.logIdentifier);
             return VertexState.INITIALIZING;
           }
-          throw new TezUncheckedException(vertex.getVertexId() + 
+          throw new TezUncheckedException(vertex.getVertexId() +
           " has -1 tasks but does not have input initializers, " +
           "1-1 uninited sources or custom vertex manager to set it at runtime");
         }
@@ -2676,14 +2696,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
               vertex.appContext.getTaskScheduler().getNumClusterNodes(),
               vertex.getTaskResource(),
               vertex.appContext.getTaskScheduler().getTotalResources());
-          List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+          List<RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
           inputList = Lists.newArrayListWithCapacity(vertex.inputsWithInitializers.size());
           for (String inputName : vertex.inputsWithInitializers) {
             inputList.add(vertex.rootInputDescriptors.get(inputName));
           }
           LOG.info("Starting root input initializers: "
               + vertex.inputsWithInitializers.size());
-          // special case when numTasks>0 and still we want to stay in initializing 
+          // special case when numTasks>0 and still we want to stay in initializing
           // state. This is handled in RootInputInitializedTransition specially.
           vertex.initWaitsForRootInitializers = true;
           vertex.rootInputInitializerManager.runInputInitializers(inputList);
@@ -2711,7 +2731,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       Resource vertexTaskResource, Resource totalResource) {
     return new RootInputInitializerManager(this, appContext, this.dagUgi, this.stateChangeNotifier);
   }
-  
+
   private boolean initializeVertexInInitializingState() {
     boolean isInitialized = initializeVertex();
     if (!isInitialized) {
@@ -2721,7 +2741,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
     return true;
   }
-  
+
   void startIfPossible() {
     if (startSignalPending) {
       // Trigger a start event to ensure route events are seen before
@@ -2735,7 +2755,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
   public static class VertexInitializedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-    
+
     static VertexState doTransition(VertexImpl vertex) {
       Preconditions.checkState(vertex.canInitVertex(), "Vertex: " + vertex.logIdentifier);
       boolean isInitialized = vertex.initializeVertexInInitializingState();
@@ -2744,15 +2764,15 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       }
 
       vertex.startIfPossible();
-      return VertexState.INITED;      
+      return VertexState.INITED;
     }
-    
+
     @Override
     public VertexState transition(VertexImpl vertex, VertexEvent event) {
       return doTransition(vertex);
     }
   }
-  
+
   // present in most transitions so that the initializer thread can be shutdown properly
   public static class RootInputInitializedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
@@ -2773,15 +2793,15 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         // All inputs initialized, shutdown the initializer.
         vertex.rootInputInitializerManager.shutdown();
       }
-      
+
       // done. check if we need to do the initialization
-      if (vertex.getState() == VertexState.INITIALIZING && 
+      if (vertex.getState() == VertexState.INITIALIZING &&
           vertex.initWaitsForRootInitializers) {
         // set the wait flag to false
         vertex.initWaitsForRootInitializers = false;
         // initialize vertex if possible and needed
         if (vertex.canInitVertex()) {
-          Preconditions.checkState(vertex.numTasks >= 0, 
+          Preconditions.checkState(vertex.numTasks >= 0,
               "Parallelism should have been set by now for vertex: " + vertex.logIdentifier);
           return VertexInitializedTransition.doTransition(vertex);
         }
@@ -2795,10 +2815,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
     @Override
     public VertexState transition(VertexImpl vertex, VertexEvent event) {
-      VertexEventOneToOneSourceSplit splitEvent = 
+      VertexEventOneToOneSourceSplit splitEvent =
           (VertexEventOneToOneSourceSplit)event;
       TezVertexID originalSplitSource = splitEvent.getOriginalSplitSource();
-      
+
       if (vertex.originalOneToOneSplitSource != null) {
         VertexState state = vertex.getState();
         Preconditions
@@ -2813,25 +2833,25 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         if (vertex.originalOneToOneSplitSource.equals(originalSplitSource)) {
           // ignore another split event that may have come from a different
           // path in the DAG. We have already split because of that source
-          LOG.info("Ignoring split of vertex " + vertex.getVertexId() + 
-              " because of split in vertex " + originalSplitSource + 
+          LOG.info("Ignoring split of vertex " + vertex.getVertexId() +
+              " because of split in vertex " + originalSplitSource +
               " sent by vertex " + splitEvent.getSenderVertex() +
               " numTasks " + splitEvent.getNumTasks());
           return state;
         }
         // cannot split from multiple sources
-        throw new TezUncheckedException("Vertex: " + vertex.getVertexId() + 
-            " asked to split by: " + originalSplitSource + 
+        throw new TezUncheckedException("Vertex: " + vertex.getVertexId() +
+            " asked to split by: " + originalSplitSource +
             " but was already split by:" + vertex.originalOneToOneSplitSource);
       }
-      
-      LOG.info("Splitting vertex " + vertex.getVertexId() + 
-          " because of split in vertex " + originalSplitSource + 
+
+      LOG.info("Splitting vertex " + vertex.getVertexId() +
+          " because of split in vertex " + originalSplitSource +
           " sent by vertex " + splitEvent.getSenderVertex() +
           " numTasks " + splitEvent.getNumTasks());
       vertex.originalOneToOneSplitSource = originalSplitSource;
       vertex.setParallelism(splitEvent.getNumTasks(), null, null, null);
-      if (vertex.getState() == VertexState.RUNNING || 
+      if (vertex.getState() == VertexState.RUNNING ||
           vertex.getState() == VertexState.INITED) {
         return vertex.getState();
       } else {
@@ -2861,19 +2881,19 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       }
       vertex.numStartedSourceVertices++;
       LOG.info("Source vertex started: " + startEvent.getSourceVertexId() +
-          " for vertex: " + vertex.getVertexId() + " numStartedSources: " + 
+          " for vertex: " + vertex.getVertexId() + " numStartedSources: " +
           vertex.numStartedSourceVertices + " numSources: " + vertex.sourceVertices.size());
-      
+
       if (vertex.numStartedSourceVertices < vertex.sourceVertices.size()) {
         LOG.info("Cannot start vertex: " + vertex.logIdentifier + " numStartedSources: "
             + vertex.numStartedSourceVertices + " numSources: " + vertex.sourceVertices.size());
         return;
       }
-      
-      // vertex meets external start dependency conditions. Save this signal in 
+
+      // vertex meets external start dependency conditions. Save this signal in
       // case we are not ready to start now and need to start later
       vertex.startSignalPending = true;
-      
+
       if (vertex.getState() != VertexState.INITED) {
         // vertex itself is not ready to start. External dependencies have already
         // notified us.
@@ -2883,14 +2903,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
             + vertex.uninitializedEdges.size());
         return;
       }
-      
+
       // vertex is inited and all dependencies are ready. Inited vertex means
       // parallelism must be set already and edges defined
       Preconditions.checkState(
           (vertex.numTasks >= 0 && vertex.uninitializedEdges.isEmpty()),
           "Cannot start vertex that is not completely defined. Vertex: "
               + vertex.logIdentifier + " numTasks: " + vertex.numTasks);
-      
+
       vertex.startIfPossible();
     }
   }
@@ -2906,14 +2926,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         + " initWaitsForRootInitializers: " + initWaitsForRootInitializers);
     return false;
   }
-  
-  public static class StartWhileInitializingTransition implements 
+
+  public static class StartWhileInitializingTransition implements
     SingleArcTransition<VertexImpl, VertexEvent> {
 
     @Override
     public void transition(VertexImpl vertex, VertexEvent event) {
       // vertex state machine does not start itself in the initializing state
-      // this start event can only come directly from the DAG. That means this 
+      // this start event can only come directly from the DAG. That means this
       // is a top level vertex of the dag
       Preconditions.checkState(
           (vertex.sourceVertices == null || vertex.sourceVertices.isEmpty()),
@@ -2926,10 +2946,10 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
   public static class StartTransition implements
     MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
-  
+
   @Override
   public VertexState transition(VertexImpl vertex, VertexEvent event) {
-      Preconditions.checkState(vertex.getState() == VertexState.INITED, 
+      Preconditions.checkState(vertex.getState() == VertexState.INITED,
           "Unexpected state " + vertex.getState() + " for " + vertex.logIdentifier);
       vertex.startTimeRequested = vertex.clock.getTime();
       return vertex.startVertex();
@@ -2937,7 +2957,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
   }
 
   private VertexState startVertex() {
-    Preconditions.checkState(getState() == VertexState.INITED, 
+    Preconditions.checkState(getState() == VertexState.INITED,
         "Vertex must be inited " + logIdentifier);
 
     startedTime = clock.getTime();
@@ -3095,7 +3115,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
         case OWN_TASK_FAILURE: vertex.tryEnactKill(trigger, TaskTerminationCause.OTHER_TASK_FAILURE); break;
         case ROOT_INPUT_INIT_FAILURE:
         case COMMIT_FAILURE:
-        case INVALID_NUM_OF_TASKS: 
+        case INVALID_NUM_OF_TASKS:
         case INIT_FAILURE:
         case INTERNAL_ERROR:
         case OTHER_VERTEX_FAILURE: vertex.tryEnactKill(trigger, TaskTerminationCause.OTHER_VERTEX_FAILURE); break;
@@ -3172,7 +3192,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       if (taskEvent.getState() == TaskState.SUCCEEDED) {
         taskSucceeded(vertex, task);
       } else if (taskEvent.getState() == TaskState.FAILED) {
-        LOG.info("Failing vertex: " + vertex.logIdentifier + 
+        LOG.info("Failing vertex: " + vertex.logIdentifier +
             " because task failed: " + taskEvent.getTaskID());
         vertex.tryEnactKill(VertexTerminationCause.OWN_TASK_FAILURE, TaskTerminationCause.OTHER_TASK_FAILURE);
         forceTransitionToKillWait = true;
@@ -3220,7 +3240,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       vertex.succeededTaskCount--;
     }
   }
-  
+
   private static class VertexNoTasksCompletedTransition implements
       MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
 
@@ -3229,7 +3249,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       return VertexImpl.checkVertexForCompletion(vertex);
     }
   }
-  
+
   private static class TaskCompletedAfterVertexSuccessTransition implements
     MultipleArcTransition<VertexImpl, VertexEvent, VertexState> {
     @Override
@@ -3241,12 +3261,12 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
       if (vEvent.getState() == TaskState.FAILED) {
         finalState = VertexState.FAILED;
         finalStatus = VertexStatus.State.FAILED;
-        diagnosticMsg = "Vertex " + vertex.logIdentifier +" failed as task " + vEvent.getTaskID() + 
+        diagnosticMsg = "Vertex " + vertex.logIdentifier +" failed as task " + vEvent.getTaskID() +
           " failed after vertex succeeded.";
       } else {
         finalState = VertexState.ERROR;
         finalStatus = VertexStatus.State.ERROR;
-        diagnosticMsg = "Vertex " + vertex.logIdentifier + " error as task " + vEvent.getTaskID() + 
+        diagnosticMsg = "Vertex " + vertex.logIdentifier + " error as task " + vEvent.getTaskID() +
             " completed with state " + vEvent.getState() + " after vertex succeeded.";
       }
       LOG.info(diagnosticMsg);
@@ -3599,14 +3619,14 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
 
   @Nullable
   @Override
-  public Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+  public Map<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
     getAdditionalInputs() {
     return this.rootInputDescriptors;
   }
 
   @Nullable
   @Override
-  public Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>> 
+  public Map<String, RootInputLeafOutput<OutputDescriptor, OutputCommitterDescriptor>>
     getAdditionalOutputs() {
     return this.additionalOutputs;
   }
@@ -3696,7 +3716,7 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     inputSpecList = new ArrayList<InputSpec>(this.getInputVerticesCount()
         + (rootInputDescriptors == null ? 0 : rootInputDescriptors.size()));
     if (rootInputDescriptors != null) {
-      for (Entry<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>> 
+      for (Entry<String, RootInputLeafOutput<InputDescriptor, InputInitializerDescriptor>>
            rootInputDescriptorEntry : rootInputDescriptors.entrySet()) {
         inputSpecList.add(new InputSpec(rootInputDescriptorEntry.getKey(),
             rootInputDescriptorEntry.getValue().getIODescriptor(), rootInputSpecs.get(
@@ -3729,18 +3749,18 @@ public class VertexImpl implements org.apache.tez.dag.app.dag.Vertex,
     }
     return outputSpecList;
   }
-  
+
   //TODO Eventually remove synchronization.
   @Override
   public synchronized List<GroupInputSpec> getGroupInputSpecList(int taskIndex) {
     return groupInputSpecList;
   }
-  
+
   @Override
   public synchronized void addSharedOutputs(Set<String> outputs) {
     this.sharedOutputs.addAll(outputs);
   }
-  
+
   @Override
   public synchronized Set<String> getSharedOutputs() {
     return this.sharedOutputs;

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
index ec05815..aba4fd9 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
@@ -111,7 +111,7 @@ public class TestDAGImpl {
   private static final Log LOG = LogFactory.getLog(TestDAGImpl.class);
   private DAGPlan dagPlan;
   private TezDAGID dagId;
-  private Configuration conf;
+  private static Configuration conf;
   private DrainDispatcher dispatcher;
   private Credentials fsTokens;
   private AppContext appContext;
@@ -344,7 +344,7 @@ public class TestDAGImpl {
   }
 
   // Create a plan with 3 vertices: A, B, C. Group(A,B)->C
-  private DAGPlan createGroupDAGPlan() {
+  static DAGPlan createGroupDAGPlan() {
     LOG.info("Setting up group dag plan");
     int dummyTaskCount = 1;
     Resource dummyTaskResource = Resource.newInstance(1, 1);
@@ -381,7 +381,7 @@ public class TestDAGImpl {
     return dag.createDag(conf);
   }
 
-  private DAGPlan createTestDAGPlan() {
+  public static DAGPlan createTestDAGPlan() {
     LOG.info("Setting up dag plan");
     DAGPlan dag = DAGPlan.newBuilder()
         .setName("testverteximpl")

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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
new file mode 100644
index 0000000..da0186e
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGRecovery.java
@@ -0,0 +1,514 @@
+/**
+ * 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.*;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.List;
+
+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.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.records.DAGProtos.DAGPlan;
+import org.apache.tez.dag.app.AppContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.DAGState;
+import org.apache.tez.dag.app.dag.VertexState;
+import org.apache.tez.dag.app.dag.event.DAGAppMasterEventDAGFinished;
+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.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.history.events.DAGInitializedEvent;
+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.records.TezDAGID;
+import org.junit.Before;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class TestDAGRecovery {
+
+  private DAGImpl dag;
+  private EventHandler mockEventHandler;
+
+  private String user = "root";
+  private String dagName = "dag1";
+
+  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;
+
+  @Before
+  public void setUp() {
+
+    mockAppContext = mock(AppContext.class, RETURNS_DEEP_STUBS);
+    when(mockAppContext.getCurrentDAG().getDagUGI()).thenReturn(null);
+    mockEventHandler = mock(EventHandler.class);
+
+    DAGPlan dagPlan = TestDAGImpl.createTestDAGPlan();
+    dag =
+        new DAGImpl(dagId, new Configuration(), dagPlan, mockEventHandler,
+            mock(TaskAttemptListener.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));
+    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, "", new TezCounters(), user, dagName));
+    assertEquals(finishTime, dag.finishTime);
+    assertFalse(dag.recoveryCommitInProgress);
+    assertEquals(finalState, recoveredState);
+  }
+
+  /**
+   * New -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_FromNew() {
+    assertNewState();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+    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());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_FromInited() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+
+    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());
+    }
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_FromStarted() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+
+    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());
+    }
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAGFinishedEvent (SUCCEEDED) -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_SUCCEEDED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    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());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAGFinishedEvent(FAILED) -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_FAILED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.FAILED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.FAILED, dag.getState());
+
+    // 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());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
+   * restoreFromDAGFinishedEvent -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_KILLED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.KILLED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.KILLED, dag.getState());
+
+    // 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());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.KILLED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent -> ->
+   * restoreFromDAGFinishedEvent -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_Finished_ERROR() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGFinishedEvent(DAGState.ERROR);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.ERROR, dag.getState());
+
+    // 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());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.ERROR, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAG_COMMIT_STARTED -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_COMMIT_STARTED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromDAGCommitStartedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.FAILED, dag.getState());
+
+    // 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());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromDAG_COMMIT_STARTED -> -> restoreFromDAGFinished (SUCCEEDED)->
+   * RecoverTransition
+   */
+  @Test
+  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());
+
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_GROUP_COMMIT_STARTED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+    restoreFromVertexGroupCommitStartedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.FAILED, dag.getState());
+
+    // 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());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.FAILED, dagFinishedEvent.getDAGState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
+   * RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_GROUP_COMMIT_STARTED_FINISHED() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+
+    restoreFromVertexGroupCommitStartedEvent();
+    restoreFromVertexGroupCommitFinishedEvent();
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+    assertEquals(DAGState.RUNNING, dag.getState());
+
+    // send recover event to 2 root vertex
+    verify(mockEventHandler, times(2)).handle(
+        any(VertexEventRecoverVertex.class));
+    assertEquals(DAGState.RUNNING, dag.getState());
+  }
+
+  /**
+   * restoreFromDAGInitializedEvent -> restoreFromDAGStartedEvent ->
+   * restoreFromVERTEX_GROUP_COMMIT_STARTED -> VERTEX_GROUP_COMMIT_FINISHED ->
+   * restoreFromDAG_Finished -> RecoverTransition
+   */
+  @Test
+  public void testDAGRecovery_GROUP_COMMIT_Finished() {
+    assertNewState();
+    restoreFromDAGInitializedEvent();
+    restoreFromDAGStartedEvent();
+
+    restoreFromVertexGroupCommitStartedEvent();
+    restoreFromVertexGroupCommitFinishedEvent();
+    restoreFromDAGFinishedEvent(DAGState.SUCCEEDED);
+
+    dag.handle(new DAGEventRecoverEvent(dagId, new ArrayList<URL>()));
+
+    // 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());
+    }
+
+    // send DAGAppMasterEventDAGFinished at last
+    assertTrue(events.get(i) instanceof DAGAppMasterEventDAGFinished);
+    DAGAppMasterEventDAGFinished dagFinishedEvent =
+        (DAGAppMasterEventDAGFinished) events.get(i);
+    assertEquals(DAGState.SUCCEEDED, dagFinishedEvent.getDAGState());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/tez/blob/f65e65ae/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
new file mode 100644
index 0000000..3b04cf6
--- /dev/null
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestTaskAttemptRecovery.java
@@ -0,0 +1,178 @@
+/**
+ * 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.fail;
+import static org.mockito.Matchers.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.never;
+import static org.mockito.Mockito.times;
+import static org.mockito.Mockito.verify;
+
+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.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.app.AppContext;
+import org.apache.tez.dag.app.ContainerContext;
+import org.apache.tez.dag.app.TaskAttemptListener;
+import org.apache.tez.dag.app.TaskHeartbeatHandler;
+import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+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.history.events.TaskAttemptFinishedEvent;
+import org.apache.tez.dag.history.events.TaskAttemptStartedEvent;
+import org.apache.tez.dag.records.TezTaskAttemptID;
+import org.apache.tez.dag.records.TezTaskID;
+import org.junit.Before;
+import org.junit.Test;
+
+@SuppressWarnings({ "unchecked", "rawtypes" })
+public class TestTaskAttemptRecovery {
+
+  private TaskAttemptImpl ta;
+  private EventHandler mockEventHandler;
+  private long startTime = System.currentTimeMillis();
+  private long finishTime = startTime + 5000;
+
+  private TezTaskAttemptID taId = mock(TezTaskAttemptID.class);
+  private String vertexName = "v1";
+
+  @Before
+  public void setUp() {
+    mockEventHandler = mock(EventHandler.class);
+    TezTaskID taskId =
+        TezTaskID.fromString("task_1407371892933_0001_1_00_000000");
+    ta =
+        new TaskAttemptImpl(taskId, 0, mockEventHandler,
+            mock(TaskAttemptListener.class), new Configuration(),
+            new SystemClock(), mock(TaskHeartbeatHandler.class),
+            mock(AppContext.class), false, Resource.newInstance(1, 1),
+            mock(ContainerContext.class), false);
+  }
+
+  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);
+
+    TaskAttemptState recoveredState =
+        ta.restoreFromEvent(new TaskAttemptFinishedEvent(taId, vertexName,
+            startTime, finishTime, state, diag, counters));
+    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);
+  }
+
+  /**
+   * No any event to restore -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_NEW() {
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+    verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+  }
+
+  /**
+   * restoreFromTAStartEvent -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_START() {
+    restoreFromTAStartEvent();
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+    verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+  }
+
+  /**
+   * restoreFromTAStartEvent -> restoreFromTAFinished (SUCCEED)
+   * -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_SUCCEED() {
+    restoreFromTAStartEvent();
+    restoreFromTAFinishedEvent(TaskAttemptState.SUCCEEDED);
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.SUCCEEDED, ta.getInternalState());
+    verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+  }
+
+  /**
+   * restoreFromTAStartEvent -> restoreFromTAFinished (KILLED)
+   * -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_KIILED() {
+    restoreFromTAStartEvent();
+    restoreFromTAFinishedEvent(TaskAttemptState.KILLED);
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
+    verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+  }
+
+  /**
+   * restoreFromTAStartEvent -> restoreFromTAFinished (FAILED)
+   * -> RecoverTransition
+   */
+  @Test
+  public void testTARecovery_FAILED() {
+    restoreFromTAStartEvent();
+    restoreFromTAFinishedEvent(TaskAttemptState.FAILED);
+
+    ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
+    assertEquals(TaskAttemptStateInternal.FAILED, ta.getInternalState());
+    verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+  }
+
+  /**
+   * restoreFromTAFinishedEvent ( no TAStartEvent before TAFinishedEvent )
+   */
+  @Test
+  public void testRecover_FINISH_BUT_NO_START() {
+    try {
+      restoreFromTAFinishedEvent(TaskAttemptState.SUCCEEDED);
+      fail("Should fail due to no TaskAttemptStartEvent before TaskAttemptFinishedEvent");
+    } catch (Throwable e) {
+      assertEquals("Finished Event seen but"
+          + " no Started Event was encountered earlier", e.getMessage());
+    }
+  }
+}


[06/25] git commit: TEZ-853. Support counters recovery. (Jeff Zhang via hitesh)

Posted by je...@apache.org.
TEZ-853. Support counters recovery. (Jeff Zhang via hitesh)


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

Branch: refs/heads/TEZ-8
Commit: 16a0f57952d554e95ae1346ac3f360998ac7faac
Parents: fcc7426
Author: Hitesh Shah <hi...@apache.org>
Authored: Thu Sep 11 22:03:26 2014 -0700
Committer: Hitesh Shah <hi...@apache.org>
Committed: Thu Sep 11 22:03:26 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/tez/common/counters/DAGCounter.java  |  1 +
 .../apache/tez/dag/api/client/DAGStatus.java    |  2 +-
 .../tez/dag/app/dag/impl/TaskAttemptImpl.java   | 53 +++++------
 .../dag/history/events/TaskFinishedEvent.java   | 10 +--
 .../dag/history/events/VertexFinishedEvent.java |  8 --
 tez-dag/src/main/proto/HistoryEvents.proto      |  4 +-
 .../tez/dag/app/dag/impl/TestTaskAttempt.java   | 16 ++++
 .../app/dag/impl/TestTaskAttemptRecovery.java   | 60 +++++++++++--
 .../TestHistoryEventsProtoConversion.java       |  6 --
 .../org/apache/tez/test/TestAMRecovery.java     | 94 +++++++++++++++++---
 11 files changed, 178 insertions(+), 77 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 4fc7e83..519aaa6 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -10,6 +10,7 @@ ALL CHANGES:
   TEZ-1544. Link to release artifacts for 0.5.0 does not point to a specific link for 0.5.0.
   TEZ-1559. Add system tests for AM recovery.
   TEZ-850. Recovery unit tests.
+  TEZ-853. Support counters recovery.
 
 Release 0.5.1: Unreleased
 

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
index 3598572..b5bdffb 100644
--- a/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
+++ b/tez-api/src/main/java/org/apache/tez/common/counters/DAGCounter.java
@@ -27,6 +27,7 @@ import org.apache.hadoop.classification.InterfaceStability;
 public enum DAGCounter {
   NUM_FAILED_TASKS, 
   NUM_KILLED_TASKS,
+  NUM_SUCCEEDED_TASKS,
   TOTAL_LAUNCHED_TASKS,
   OTHER_LOCAL_TASKS,
   DATA_LOCAL_TASKS,

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
index 297c6f2..7c51c42 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/client/DAGStatus.java
@@ -170,7 +170,7 @@ public class DAGStatus {
       + ", diagnostics="
       + StringUtils.join(getDiagnostics(), LINE_SEPARATOR)
       + ", counters="
-      + (dagCounters == null ? "null" : dagCounters.toString()));
+      + (getDAGCounters() == null ? "null" : getDAGCounters().toString()));
     return sb.toString();
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/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 7ba90b5..b8430cc 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
@@ -791,6 +791,7 @@ public class TaskAttemptImpl implements TaskAttempt,
         this.launchTime = tEvent.getStartTime();
         recoveryStartEventSeen = true;
         recoveredState = TaskAttemptState.RUNNING;
+        sendEvent(createDAGCounterUpdateEventTALaunched(this));
         return recoveredState;
       }
       case TASK_ATTEMPT_FINISHED:
@@ -806,6 +807,7 @@ public class TaskAttemptImpl implements TaskAttempt,
         this.reportedStatus.state = tEvent.getState();
         this.diagnostics.add(tEvent.getDiagnostics());
         this.recoveredState = tEvent.getState();
+        sendEvent(createDAGCounterUpdateEventTAFinished(this, tEvent.getState()));
         return recoveredState;
       }
       default:
@@ -829,47 +831,29 @@ public class TaskAttemptImpl implements TaskAttempt,
   }
 
   // TOOD Merge some of these JobCounter events.
-  private static DAGEventCounterUpdate createJobCounterUpdateEventTALaunched(
+  private static DAGEventCounterUpdate createDAGCounterUpdateEventTALaunched(
       TaskAttemptImpl ta) {
-    DAGEventCounterUpdate jce =
+    DAGEventCounterUpdate dagCounterEvent =
         new DAGEventCounterUpdate(
             ta.getDAGID()
             );
-    jce.addCounterUpdate(DAGCounter.TOTAL_LAUNCHED_TASKS, 1);
-    return jce;
+    dagCounterEvent.addCounterUpdate(DAGCounter.TOTAL_LAUNCHED_TASKS, 1);
+    return dagCounterEvent;
   }
 
-  private static DAGEventCounterUpdate createJobCounterUpdateEventSlotMillis(
-      TaskAttemptImpl ta) {
+  private static DAGEventCounterUpdate createDAGCounterUpdateEventTAFinished(
+      TaskAttemptImpl taskAttempt, TaskAttemptState taState) {
     DAGEventCounterUpdate jce =
-        new DAGEventCounterUpdate(
-            ta.getDAGID()
-            );
+        new DAGEventCounterUpdate(taskAttempt.getDAGID());
 
-//    long slotMillis = computeSlotMillis(ta);
-//    jce.addCounterUpdate(DAGCounter.SLOTS_MILLIS_TASKS, slotMillis);
-    return jce;
-  }
-
-  private static DAGEventCounterUpdate createJobCounterUpdateEventTATerminated(
-      TaskAttemptImpl taskAttempt, boolean taskAlreadyCompleted,
-      TaskAttemptStateInternal taState) {
-    DAGEventCounterUpdate jce =
-        new DAGEventCounterUpdate(
-            taskAttempt.getDAGID());
-
-    if (taState == TaskAttemptStateInternal.FAILED) {
+    if (taState == TaskAttemptState.FAILED) {
       jce.addCounterUpdate(DAGCounter.NUM_FAILED_TASKS, 1);
-    } else if (taState == TaskAttemptStateInternal.KILLED) {
+    } else if (taState == TaskAttemptState.KILLED) {
       jce.addCounterUpdate(DAGCounter.NUM_KILLED_TASKS, 1);
+    } else if (taState == TaskAttemptState.SUCCEEDED ) {
+      jce.addCounterUpdate(DAGCounter.NUM_SUCCEEDED_TASKS, 1);
     }
 
-//    long slotMillisIncrement = computeSlotMillis(taskAttempt);
-//    if (!taskAlreadyCompleted) {
-//      // dont double count the elapsed time
-//      jce.addCounterUpdate(DAGCounter.SLOTS_MILLIS_TASKS, slotMillisIncrement);
-//    }
-
     return jce;
   }
 
@@ -1142,8 +1126,8 @@ public class TaskAttemptImpl implements TaskAttempt,
         ta.addDiagnosticInfo(((DiagnosableEvent) event).getDiagnosticInfo());
       }
 
-      ta.sendEvent(createJobCounterUpdateEventTATerminated(ta, false,
-          helper.getTaskAttemptStateInternal()));
+      ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
+          helper.getTaskAttemptState()));
       if (ta.getLaunchTime() != 0) {
         // TODO For cases like this, recovery goes for a toss, since the the
         // attempt will not exist in the history file.
@@ -1184,7 +1168,7 @@ public class TaskAttemptImpl implements TaskAttempt,
           .createSocketAddr(ta.nodeHttpAddress); // TODO: Costly?
       ta.trackerName = StringInterner.weakIntern(nodeHttpInetAddr.getHostName());
       ta.httpPort = nodeHttpInetAddr.getPort();
-      ta.sendEvent(createJobCounterUpdateEventTALaunched(ta));
+      ta.sendEvent(createDAGCounterUpdateEventTALaunched(ta));
 
       LOG.info("TaskAttempt: [" + ta.attemptId + "] started."
           + " Is using containerId: [" + ta.containerId + "]" + " on NM: ["
@@ -1316,7 +1300,8 @@ public class TaskAttemptImpl implements TaskAttempt,
       ta.setFinishTime();
       // Send out history event.
       ta.logJobHistoryAttemptFinishedEvent(TaskAttemptStateInternal.SUCCEEDED);
-      ta.sendEvent(createJobCounterUpdateEventSlotMillis(ta));
+      ta.sendEvent(createDAGCounterUpdateEventTAFinished(ta,
+          TaskAttemptState.SUCCEEDED));
 
       // Inform the Scheduler.
       ta.sendEvent(new AMSchedulerEventTAEnded(ta, ta.containerId,
@@ -1415,6 +1400,8 @@ public class TaskAttemptImpl implements TaskAttempt,
           // TODO abort taskattempt
           taskAttempt.sendEvent(new TaskEventTAUpdate(taskAttempt.attemptId,
               TaskEventType.T_ATTEMPT_KILLED));
+          taskAttempt.sendEvent(createDAGCounterUpdateEventTAFinished(taskAttempt,
+              getExternalState(TaskAttemptStateInternal.KILLED)));
           endState = TaskAttemptStateInternal.KILLED;
           break;
         case SUCCEEDED:

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
index c60ef58..9323270 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/TaskFinishedEvent.java
@@ -25,7 +25,6 @@ import java.io.OutputStream;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.api.oldrecords.TaskState;
 import org.apache.tez.dag.history.HistoryEvent;
 import org.apache.tez.dag.history.HistoryEventType;
@@ -45,7 +44,7 @@ public class TaskFinishedEvent implements HistoryEvent {
   private TezCounters tezCounters;
   private TezTaskAttemptID successfulAttemptID;
   private String diagnostics;
-  
+
   public TaskFinishedEvent(TezTaskID taskID,
       String vertexName, long startTime, long finishTime,
       TezTaskAttemptID successfulAttemptID,
@@ -85,9 +84,6 @@ public class TaskFinishedEvent implements HistoryEvent {
     if (diagnostics != null) {
       builder.setDiagnostics(diagnostics);
     }
-    if (tezCounters != null) {
-      builder.setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters));
-    }
     if (successfulAttemptID != null) {
       builder.setSuccessfulTaskAttemptId(successfulAttemptID.toString());
     }
@@ -101,10 +97,6 @@ public class TaskFinishedEvent implements HistoryEvent {
     if (proto.hasDiagnostics()) {
       this.diagnostics = proto.getDiagnostics();
     }
-    if (proto.hasCounters()) {
-      this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
-          proto.getCounters());
-    }
     if (proto.hasSuccessfulTaskAttemptId()) {
       this.successfulAttemptID =
           TezTaskAttemptID.fromString(proto.getSuccessfulTaskAttemptId());

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
index df3551a..8057714 100644
--- a/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
+++ b/tez-dag/src/main/java/org/apache/tez/dag/history/events/VertexFinishedEvent.java
@@ -25,7 +25,6 @@ import java.io.OutputStream;
 import org.apache.commons.logging.Log;
 import org.apache.commons.logging.LogFactory;
 import org.apache.tez.common.counters.TezCounters;
-import org.apache.tez.dag.api.DagTypeConverters;
 import org.apache.tez.dag.app.dag.VertexState;
 import org.apache.tez.dag.app.dag.impl.VertexStats;
 import org.apache.tez.dag.history.HistoryEvent;
@@ -99,9 +98,6 @@ public class VertexFinishedEvent implements HistoryEvent, SummaryEvent {
     if (diagnostics != null) {
       builder.setDiagnostics(diagnostics);
     }
-    if (tezCounters != null) {
-      builder.setCounters(DagTypeConverters.convertTezCountersToProto(tezCounters));
-    }
     return builder.build();
   }
 
@@ -113,10 +109,6 @@ public class VertexFinishedEvent implements HistoryEvent, SummaryEvent {
     if (proto.hasDiagnostics())  {
       this.diagnostics = proto.getDiagnostics();
     }
-    if (proto.hasCounters()) {
-      this.tezCounters = DagTypeConverters.convertTezCountersFromProto(
-          proto.getCounters());
-    }
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/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 5cbe540..821612a 100644
--- a/tez-dag/src/main/proto/HistoryEvents.proto
+++ b/tez-dag/src/main/proto/HistoryEvents.proto
@@ -136,7 +136,6 @@ message VertexFinishedProto {
   optional int64 finish_time = 3;
   optional int32 state = 4;
   optional string diagnostics = 5;
-  optional TezCountersProto counters = 6;
 }
 
 message TaskStartedProto {
@@ -150,8 +149,7 @@ message TaskFinishedProto {
   optional int64 finish_time = 2;
   optional int32 state = 3;
   optional string diagnostics = 4;
-  optional TezCountersProto counters = 5;
-  optional string successful_task_attempt_id = 6;
+  optional string successful_task_attempt_id = 5;
 }
 
 message TaskAttemptStartedProto {

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/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 3633a85..8e134f2 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
@@ -67,6 +67,7 @@ import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
 import org.apache.tez.dag.app.dag.Vertex;
 import org.apache.tez.dag.app.dag.event.DAGEvent;
+import org.apache.tez.dag.app.dag.event.DAGEventCounterUpdate;
 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.TaskAttemptEventContainerTerminated;
@@ -423,6 +424,9 @@ public class TestTaskAttempt {
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     taImpl.handle(new TaskAttemptEventContainerTerminated(taskAttemptID,
         "Terminated"));
@@ -571,6 +575,9 @@ public class TestTaskAttempt {
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     taImpl.handle(new TaskAttemptEventContainerTerminated(taskAttemptID,
         "Terminated"));
@@ -658,6 +665,9 @@ public class TestTaskAttempt {
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEventsAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEventsAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     taImpl.handle(new TaskAttemptEvent(taskAttemptID,
         TaskAttemptEventType.TA_CONTAINER_TERMINATED_BY_SYSTEM));
@@ -744,6 +754,9 @@ public class TestTaskAttempt {
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     // Send out a Node Failure.
     taImpl.handle(new TaskAttemptEventNodeFailed(taskAttemptID, "NodeDecomissioned"));
@@ -838,6 +851,9 @@ public class TestTaskAttempt {
     verifyEventType(
         arg.getAllValues().subList(expectedEventsAtRunning,
             expectedEvenstAfterTerminating), AMSchedulerEventTAEnded.class, 1);
+    verifyEventType(
+        arg.getAllValues().subList(expectedEventsAtRunning,
+            expectedEvenstAfterTerminating), DAGEventCounterUpdate.class, 1);
 
     // Send out a Node Failure.
     taImpl.handle(new TaskAttemptEventNodeFailed(taskAttemptID, "NodeDecomissioned"));

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/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
index 3b04cf6..a443a35 100644
--- 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
@@ -20,16 +20,17 @@ package org.apache.tez.dag.app.dag.impl;
 
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.fail;
-import static org.mockito.Matchers.any;
 import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.never;
 import static org.mockito.Mockito.times;
 import static org.mockito.Mockito.verify;
 
+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;
@@ -39,6 +40,7 @@ import org.apache.tez.dag.app.ContainerContext;
 import org.apache.tez.dag.app.TaskAttemptListener;
 import org.apache.tez.dag.app.TaskHeartbeatHandler;
 import org.apache.tez.dag.app.dag.TaskAttemptStateInternal;
+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;
@@ -48,6 +50,7 @@ import org.apache.tez.dag.records.TezTaskAttemptID;
 import org.apache.tez.dag.records.TezTaskID;
 import org.junit.Before;
 import org.junit.Test;
+import org.mockito.ArgumentCaptor;
 
 @SuppressWarnings({ "unchecked", "rawtypes" })
 public class TestTaskAttemptRecovery {
@@ -98,6 +101,17 @@ public class TestTaskAttemptRecovery {
     assertEquals(state, recoveredState);
   }
 
+  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
    */
@@ -105,7 +119,14 @@ public class TestTaskAttemptRecovery {
   public void testTARecovery_NEW() {
     ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
     assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
-    verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+
+    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);
   }
 
   /**
@@ -117,7 +138,14 @@ public class TestTaskAttemptRecovery {
 
     ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
     assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
-    verify(mockEventHandler, times(1)).handle(any(TaskEventTAUpdate.class));
+
+    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);
   }
 
   /**
@@ -131,7 +159,13 @@ public class TestTaskAttemptRecovery {
 
     ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
     assertEquals(TaskAttemptStateInternal.SUCCEEDED, ta.getInternalState());
-    verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+
+    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);
   }
 
   /**
@@ -145,7 +179,13 @@ public class TestTaskAttemptRecovery {
 
     ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
     assertEquals(TaskAttemptStateInternal.KILLED, ta.getInternalState());
-    verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+
+    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);
   }
 
   /**
@@ -159,7 +199,13 @@ public class TestTaskAttemptRecovery {
 
     ta.handle(new TaskAttemptEvent(taId, TaskAttemptEventType.TA_RECOVER));
     assertEquals(TaskAttemptStateInternal.FAILED, ta.getInternalState());
-    verify(mockEventHandler, never()).handle(any(TaskEventTAUpdate.class));
+
+    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);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/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 72d0c25..5bb7d35 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
@@ -354,7 +354,6 @@ public class TestHistoryEventsProtoConversion {
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(), deserializedEvent.getState());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
-      Assert.assertEquals(event.getTezCounters(), deserializedEvent.getTezCounters());
       logEvents(event, deserializedEvent);
     }
     {
@@ -370,7 +369,6 @@ public class TestHistoryEventsProtoConversion {
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(), deserializedEvent.getState());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
-      Assert.assertEquals(event.getTezCounters(), deserializedEvent.getTezCounters());
       logEvents(event, deserializedEvent);
     }
   }
@@ -403,8 +401,6 @@ public class TestHistoryEventsProtoConversion {
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(),
           deserializedEvent.getState());
-      Assert.assertEquals(event.getTezCounters(),
-          deserializedEvent.getTezCounters());
       Assert.assertEquals(event.getSuccessfulAttemptID(),
           deserializedEvent.getSuccessfulAttemptID());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());
@@ -425,8 +421,6 @@ public class TestHistoryEventsProtoConversion {
           deserializedEvent.getFinishTime());
       Assert.assertEquals(event.getState(),
           deserializedEvent.getState());
-      Assert.assertEquals(event.getTezCounters(),
-          deserializedEvent.getTezCounters());
       Assert.assertEquals(event.getSuccessfulAttemptID(),
           deserializedEvent.getSuccessfulAttemptID());
       Assert.assertEquals(event.getDiagnostics(), deserializedEvent.getDiagnostics());

http://git-wip-us.apache.org/repos/asf/tez/blob/16a0f579/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
index eae38f0..bd2fe99 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestAMRecovery.java
@@ -24,6 +24,7 @@ import java.io.IOException;
 import java.util.ArrayList;
 import java.util.EnumSet;
 import java.util.List;
+import java.util.Map;
 import java.util.Random;
 
 import org.apache.commons.logging.Log;
@@ -38,6 +39,8 @@ import org.apache.tez.client.TezClient;
 import org.apache.tez.client.TezClientUtils;
 import org.apache.tez.common.TezCommonUtils;
 import org.apache.tez.common.TezUtils;
+import org.apache.tez.common.counters.DAGCounter;
+import org.apache.tez.common.counters.TezCounters;
 import org.apache.tez.dag.api.DAG;
 import org.apache.tez.dag.api.Edge;
 import org.apache.tez.dag.api.EdgeProperty;
@@ -193,7 +196,12 @@ public class TestAMRecovery {
     DAG dag =
         createDAG(ControlledInputReadyVertexManager.class,
             DataMovementType.BROADCAST, true);
-    runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
+
     List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
     List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
 
@@ -220,7 +228,11 @@ public class TestAMRecovery {
     DAG dag =
         createDAG(ControlledInputReadyVertexManager.class,
             DataMovementType.BROADCAST, false);
-    runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
 
     List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
     List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -244,11 +256,15 @@ public class TestAMRecovery {
    * @throws Exception
    */
   @Test(timeout = 120000)
-  public void testVertexPartialComplete_One2One() throws Exception {
+  public void testVertexPartialFinished_One2One() throws Exception {
     DAG dag =
         createDAG(ControlledInputReadyVertexManager.class,
             DataMovementType.ONE_TO_ONE, true);
-    runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
 
     List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
     List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -273,11 +289,15 @@ public class TestAMRecovery {
    * @throws Exception
    */
   @Test(timeout = 120000)
-  public void testVertexCompletelyComplete_One2One() throws Exception {
+  public void testVertexCompletelyFinished_One2One() throws Exception {
     DAG dag =
         createDAG(ControlledInputReadyVertexManager.class,
             DataMovementType.ONE_TO_ONE, false);
-    runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
 
     List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
     List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -306,7 +326,11 @@ public class TestAMRecovery {
     DAG dag =
         createDAG(ControlledShuffleVertexManager.class,
             DataMovementType.SCATTER_GATHER, true);
-    runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(5, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(1, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
 
     List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
     List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -335,7 +359,11 @@ public class TestAMRecovery {
     DAG dag =
         createDAG(ControlledShuffleVertexManager.class,
             DataMovementType.SCATTER_GATHER, false);
-    runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    TezCounters counters = runDAGAndVerify(dag, DAGStatus.State.SUCCEEDED);
+    assertEquals(4, counters.findCounter(DAGCounter.TOTAL_LAUNCHED_TASKS).getValue());
+    assertEquals(0, counters.findCounter(DAGCounter.NUM_KILLED_TASKS).getValue());
+    assertEquals(4, counters.findCounter(DAGCounter.NUM_SUCCEEDED_TASKS).getValue());
+    assertEquals(2, counters.findCounter(TestCounter.Counter_1).getValue());
 
     List<HistoryEvent> historyEvents1 = readRecoveryLog(1);
     List<HistoryEvent> historyEvents2 = readRecoveryLog(2);
@@ -369,13 +397,14 @@ public class TestAMRecovery {
 
   }
 
-  void runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception {
+  TezCounters runDAGAndVerify(DAG dag, DAGStatus.State finalState) throws Exception {
     tezSession.waitTillReady();
     DAGClient dagClient = tezSession.submitDAG(dag);
     DAGStatus dagStatus =
         dagClient.waitForCompletionWithStatusUpdates(EnumSet
             .of(StatusGetOpts.GET_COUNTERS));
     Assert.assertEquals(finalState, dagStatus.getState());
+    return dagStatus.getDAGCounters();
   }
 
   /**
@@ -479,6 +508,19 @@ public class TestAMRecovery {
         }
       }
     }
+
+    @Override
+    public void onVertexStarted(Map<String, List<Integer>> completions) {
+      // sleep for 1 seconds to delay the running of task in v2.
+      // this could keep the case that task of v1 is partial finished or completely
+      // finished, and at the same time the task of v2 is not started
+      try {
+        Thread.sleep(1*1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      super.onVertexStarted(completions);
+    }
   }
 
   public static class ControlledShuffleVertexManager extends
@@ -516,6 +558,19 @@ public class TestAMRecovery {
         }
       }
     }
+
+    @Override
+    public void onVertexStarted(Map<String, List<Integer>> completions) {
+      // sleep for 1 seconds to delay the running of task in v2.
+      // this could keep the case that task of v1 is partial finished or completely
+      // finished, and at the same time the task of v2 is not started
+      try {
+        Thread.sleep(1*1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      super.onVertexStarted(completions);
+    }
   }
 
   public static class ControlledImmediateStartVertexManager extends
@@ -554,6 +609,19 @@ public class TestAMRecovery {
         }
       }
     }
+
+    @Override
+    public void onVertexStarted(Map<String, List<Integer>> completions) {
+      // sleep for 1 seconds to delay the running of task in v2.
+      // this could keep the case that task of v1 is partial finished or completely
+      // finished, and at the same time the task of v2 is not started
+      try {
+        Thread.sleep(1*1000);
+      } catch (InterruptedException e) {
+        e.printStackTrace();
+      }
+      super.onVertexStarted(completions);
+    }
   }
 
   /**
@@ -592,6 +660,10 @@ public class TestAMRecovery {
     }
   }
 
+  public static enum TestCounter {
+    Counter_1,
+  }
+
   /**
    * Do nothing if it is in task 0, sleep 3 seconds for other tasks. This enable
    * us to kill AM in VM when some tasks are still running.
@@ -605,8 +677,10 @@ public class TestAMRecovery {
 
     @Override
     public void run() throws Exception {
+      getContext().getCounters().findCounter(TestCounter.Counter_1).increment(1);
       if (getContext().getTaskIndex() == 0) {
-        return;
+        // keep task_0 running for 1 seconds to wait for task_1 start running
+        Thread.sleep(1 * 1000);;
       } else {
         Thread.sleep(3 * 1000);
       }


[08/25] TEZ-1539. Change InputInitializerEvent semantics to SEND_ONCE_ON_TASK_SUCCESS. (sseth)

Posted by je...@apache.org.
http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 04e2219..c003e05 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
@@ -75,6 +75,7 @@ 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.TezConfiguration;
+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.EdgeProperty.DataMovementType;
@@ -116,6 +117,7 @@ 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.TaskEventTAUpdate;
 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.VertexEventRootInputFailed;
@@ -633,6 +635,208 @@ public class TestVertexImpl {
     return dag;
   }
 
+  private DAGPlan createDAGPlanWithRunningInitializer3() {
+    // v2    v1 (send event to v3)
+    //  \    /
+    //   \  /
+    //   v3 -----(In)
+    //  (Receive events from v1)
+    LOG.info("Setting up dag plan with running input initializer3");
+    DAGPlan dag = DAGPlan.newBuilder()
+        .setName("DagWithInputInitializer3")
+        .addVertex(
+            VertexPlan.newBuilder()
+                .setName("vertex1")
+                .setType(PlanVertexType.NORMAL)
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(1)
+                        .setVirtualCores(1)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("x1.y1")
+                        .build()
+                )
+                .addOutEdgeId("e1")
+                .build()
+        )
+        .addVertex(
+            VertexPlan.newBuilder()
+                .setName("vertex2")
+                .setType(PlanVertexType.NORMAL)
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(1)
+                        .setVirtualCores(1)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("x1.y1")
+                        .build()
+                )
+                .addOutEdgeId("e2")
+                .build()
+        )
+        .addVertex(
+            VertexPlan.newBuilder()
+                .setName("vertex3")
+                .setType(PlanVertexType.NORMAL)
+                .addInputs(
+                    RootInputLeafOutputProto.newBuilder()
+                        .setControllerDescriptor(
+                            TezEntityDescriptorProto.newBuilder().setClassName(
+                                "IrrelevantInitializerClassName"))
+                        .setName("input1")
+                        .setIODescriptor(
+                            TezEntityDescriptorProto.newBuilder()
+                                .setClassName("InputClazz")
+                                .build()
+                        )
+                        .build()
+                )
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(20)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("x2.y2")
+                        .build()
+                )
+                .addInEdgeId("e1")
+                .addInEdgeId("e2")
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v1_v3"))
+                .setInputVertexName("vertex1")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+                .setOutputVertexName("vertex3")
+                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+                .setId("e1")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v2_v3"))
+                .setInputVertexName("vertex2")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+                .setOutputVertexName("vertex3")
+                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+                .setId("e2")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .build();
+    return dag;
+  }
+
+  private DAGPlan createDAGPlanWithRunningInitializer4() {
+    //   v1 (send event to v3)
+    //    |
+    //    |
+    //   v2   (In)    (v2 can optioanlly send events to v2. Is setup via the initializer)
+    //    |   /
+    //    |  /
+    //    v3 (Receive events from v1)
+    // Events are not generated by a directly connected vertex
+    LOG.info("Setting up dag plan with running input initializer4");
+    DAGPlan dag = DAGPlan.newBuilder()
+        .setName("DagWithInputInitializer4")
+        .addVertex(
+            VertexPlan.newBuilder()
+                .setName("vertex1")
+                .setType(PlanVertexType.NORMAL)
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(1)
+                        .setVirtualCores(1)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("x1.y1")
+                        .build()
+                )
+                .addOutEdgeId("e1")
+                .build()
+        )
+        .addVertex(
+            VertexPlan.newBuilder()
+                .setName("vertex2")
+                .setType(PlanVertexType.NORMAL)
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(2)
+                        .setVirtualCores(1)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("x1.y1")
+                        .build()
+                )
+                .addInEdgeId("e1")
+                .addOutEdgeId("e2")
+                .build()
+        )
+        .addVertex(
+            VertexPlan.newBuilder()
+                .setName("vertex3")
+                .setType(PlanVertexType.NORMAL)
+                .addInputs(
+                    RootInputLeafOutputProto.newBuilder()
+                        .setControllerDescriptor(
+                            TezEntityDescriptorProto.newBuilder().setClassName(
+                                "IrrelevantInitializerClassName"))
+                        .setName("input1")
+                        .setIODescriptor(
+                            TezEntityDescriptorProto.newBuilder()
+                                .setClassName("InputClazz")
+                                .build()
+                        )
+                        .build()
+                )
+                .setTaskConfig(
+                    PlanTaskConfiguration.newBuilder()
+                        .setNumTasks(20)
+                        .setVirtualCores(4)
+                        .setMemoryMb(1024)
+                        .setJavaOpts("")
+                        .setTaskModule("x2.y2")
+                        .build()
+                )
+                .addInEdgeId("e2")
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v1_v3"))
+                .setInputVertexName("vertex1")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+                .setOutputVertexName("vertex2")
+                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+                .setId("e1")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .addEdge(
+            EdgePlan.newBuilder()
+                .setEdgeDestination(TezEntityDescriptorProto.newBuilder().setClassName("v2_v3"))
+                .setInputVertexName("vertex2")
+                .setEdgeSource(TezEntityDescriptorProto.newBuilder().setClassName("o2"))
+                .setOutputVertexName("vertex3")
+                .setDataMovementType(PlanEdgeDataMovementType.BROADCAST)
+                .setId("e2")
+                .setDataSourceType(PlanEdgeDataSourceType.PERSISTED)
+                .setSchedulingType(PlanEdgeSchedulingType.SEQUENTIAL)
+                .build()
+        )
+        .build();
+    return dag;
+  }
+
+
   private DAGPlan createDAGPlanWithRunningInitializer() {
     LOG.info("Setting up dag plan with running input initializer");
     DAGPlan dag = DAGPlan.newBuilder()
@@ -643,7 +847,7 @@ public class TestVertexImpl {
                 .setType(PlanVertexType.NORMAL)
                 .setTaskConfig(
                     PlanTaskConfiguration.newBuilder()
-                        .setNumTasks(10)
+                        .setNumTasks(1)
                         .setVirtualCores(4)
                         .setMemoryMb(1024)
                         .setJavaOpts("")
@@ -1742,7 +1946,7 @@ public class TestVertexImpl {
     for (PlanVertexGroupInfo groupInfo : dagPlan.getVertexGroupsList()) {
       vertexGroups.put(groupInfo.getGroupName(), new VertexGroupInfo(groupInfo));
     }
-    updateTracker = new StateChangeNotifier(dag);
+    updateTracker = new StateChangeNotifier(appContext.getCurrentDAG());
     setupVertices();
     when(dag.getVertex(any(TezVertexID.class))).thenAnswer(new Answer<Vertex>() {
       @Override
@@ -2962,16 +3166,390 @@ public class TestVertexImpl {
     Assert.assertEquals(VertexState.SUCCEEDED, v1.getState());
 
     // At this point, 2 events should have been received - since the dispatcher is complete.
-    Assert.assertEquals(2, initializer.stateUpdateEvents.size());
+    Assert.assertEquals(2, initializer.stateUpdates.size());
+    Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.RUNNING,
+        initializer.stateUpdates.get(0).getVertexState());
+    Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.SUCCEEDED,
+        initializer.stateUpdates.get(1).getVertexState());
+  }
+
+  @Test(timeout = 1000000)
+  public void testInputInitializerEventMultipleAttempts() throws Exception {
+    useCustomInitializer = true;
+    customInitializer = new EventHandlingRootInputInitializer(null);
+    EventHandlingRootInputInitializer initializer =
+        (EventHandlingRootInputInitializer) customInitializer;
+    setupPreDagCreation();
+    dagPlan = createDAGPlanWithRunningInitializer4();
+    setupPostDagCreation();
+
+    VertexImplWithRunningInputInitializer v1 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+    VertexImplWithRunningInputInitializer v2 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+    VertexImplWithRunningInputInitializer v3 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+    initVertex(v1);
+    startVertex(v1);
+    dispatcher.await();
+
+    // Vertex1 start should trigger downstream vertices
+    Assert.assertEquals(VertexState.RUNNING, v1.getState());
+    Assert.assertEquals(VertexState.RUNNING, v2.getState());
+    Assert.assertEquals(VertexState.INITIALIZING, v3.getState());
+
+    ByteBuffer expected;
+
+    // Genrate events from v1 to v3's InputInitializer
+    ByteBuffer payload = ByteBuffer.allocate(12).putInt(0, 1).putInt(4, 0).putInt(8, 0);
+    InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", payload);
+    // Create taskId and taskAttemptId for the single task that exists in vertex1
+    TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+    TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+    TezEvent tezEvent = new TezEvent(event,
+        new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+    dispatcher.getEventHandler()
+        .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+    dispatcher.await();
+
+    payload = ByteBuffer.allocate(12).putInt(0, 1).putInt(4, 0).putInt(8, 1);
+    expected = payload;
+    event = InputInitializerEvent.create("vertex3", "input1", payload);
+    // Create taskId and taskAttemptId for the single task that exists in vertex1
+    TezTaskAttemptID ta1_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 1);
+    tezEvent = new TezEvent(event,
+        new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta1_t0_v1));
+    dispatcher.getEventHandler()
+        .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+    dispatcher.await();
+
+    // Events should not be cached in the vertex, since the initializer is running
+    Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+    // Events should be cached since the tasks have not succeeded.
+    // Verify that events are cached
+    RootInputInitializerManager.InitializerWrapper initializerWrapper =
+        v3.rootInputInitializerManager.getInitializerWrapper("input1");
+    Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+    Assert.assertEquals(2, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+
+    // Get all tasks of vertex1 to succeed.
+    for (TezTaskID taskId : v1.getTasks().keySet()) {
+      // Make attempt 1 of every task succeed
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 1);
+      v1.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+      v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+      dispatcher.await();
+      v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+    }
+    dispatcher.await();
+
+    // v3 would have processed an INIT event and moved into INITIALIZING state.
+    // Since source tasks were complete - the events should have been consumed.
+    // Initializer would have run, and processed events.
+    while (v3.getState()  != VertexState.RUNNING) {
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(VertexState.RUNNING, v3.getState());
+
+    Assert.assertEquals(1, initializer.initializerEvents.size());
+    Assert.assertEquals(expected, initializer.initializerEvents.get(0).getUserPayload());
+
+  }
+
+  @Test(timeout = 10000)
+  public void testInputInitializerEventsMultipleSources() throws Exception {
+    useCustomInitializer = true;
+    customInitializer = new EventHandlingRootInputInitializer(null);
+    EventHandlingRootInputInitializer initializer =
+        (EventHandlingRootInputInitializer) customInitializer;
+    initializer.setNumExpectedEvents(4);
+    setupPreDagCreation();
+    dagPlan = createDAGPlanWithRunningInitializer4();
+    setupPostDagCreation();
+
+    VertexImplWithRunningInputInitializer v1 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+    VertexImplWithRunningInputInitializer v2 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+    VertexImplWithRunningInputInitializer v3 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+    initVertex(v1);
+    startVertex(v1);
+    dispatcher.await();
+
+    // Vertex1 start should trigger downstream vertices
+    Assert.assertEquals(VertexState.RUNNING, v1.getState());
+    Assert.assertEquals(VertexState.RUNNING, v2.getState());
+    Assert.assertEquals(VertexState.INITIALIZING, v3.getState());
+
+    List<ByteBuffer> expectedPayloads = new LinkedList<ByteBuffer>();
+
+    // Genrate events from v1 to v3's InputInitializer
+    ByteBuffer payload = ByteBuffer.allocate(12).putInt(0, 1).putInt(4, 0).putInt(8, 0);
+    expectedPayloads.add(payload);
+    InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", payload);
+    // Create taskId and taskAttemptId for the single task that exists in vertex1
+    TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+    TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+    TezEvent tezEvent = new TezEvent(event,
+        new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+    dispatcher.getEventHandler()
+        .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+    dispatcher.await();
+
+    // Events should not be cached in the vertex, since the initializer is running
+    Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+    // Events should be cached since the tasks have not succeeded.
+    // Verify that events are cached
+    RootInputInitializerManager.InitializerWrapper initializerWrapper =
+        v3.rootInputInitializerManager.getInitializerWrapper("input1");
+    Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+    Assert.assertEquals(1, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+
+    // Get all tasks of vertex1 to succeed.
+    for (TezTaskID taskId : v1.getTasks().keySet()) {
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+      v1.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+      v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+      dispatcher.await();
+      v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+    }
+    dispatcher.await();
+
+    Assert.assertEquals(1, initializer.initializerEvents.size());
+
+
+    // Test written based on this
+    Assert.assertEquals(2, v2.getTotalTasks());
+    // Generate events from v2 to v3's initializer. 1 from task 0, 2 from task 1
+    for (Task task : v2.getTasks().values()) {
+      TezTaskID taskId = task.getTaskId();
+      TezTaskAttemptID attemptId = TezTaskAttemptID.getInstance(taskId, 0);
+      int numEventsFromTask = taskId.getId() + 1;
+      for (int i = 0; i < numEventsFromTask; i++) {
+        payload = ByteBuffer.allocate(12).putInt(0, 2).putInt(4, taskId.getId()).putInt(8, i);
+        expectedPayloads.add(payload);
+        InputInitializerEvent event2 = InputInitializerEvent.create("vertex3", "input1", payload);
+        TezEvent tezEvent2 = new TezEvent(event2,
+            new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex2", "vertex3", attemptId));
+        dispatcher.getEventHandler()
+            .handle(
+                new VertexEventRouteEvent(v2.getVertexId(), Collections.singletonList(tezEvent2)));
+        dispatcher.await();
+      }
+    }
+
+    // Validate queueing of these events
+    // Only v2 events pending
+    Assert.assertEquals(1, initializerWrapper.getPendingEvents().keySet().size());
+    // 3 events pending
+    Assert.assertEquals(3, initializerWrapper.getPendingEvents().get(v2.getName()).size());
+
+    // Get all tasks of vertex1 to succeed.
+    for (TezTaskID taskId : v2.getTasks().keySet()) {
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+      v2.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+      v2.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+      dispatcher.await();
+      v2.stateChangeNotifier.taskSucceeded(v2.getName(), taskId, taskAttemptId.getId());
+    }
+    dispatcher.await();
+
+    // v3 would have processed an INIT event and moved into INITIALIZING state.
+    // Since source tasks were complete - the events should have been consumed.
+    // Initializer would have run, and processed events.
+    while (v3.getState()  != VertexState.RUNNING) {
+      Thread.sleep(10);
+    }
+    Assert.assertEquals(VertexState.RUNNING, v3.getState());
+
+    Assert.assertEquals(4, initializer.initializerEvents.size());
+    Assert.assertTrue(initializer.initComplete.get());
+
+    Assert.assertEquals(2, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+    Assert.assertEquals(0, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+    for (InputInitializerEvent initializerEvent : initializer.initializerEvents) {
+      expectedPayloads.remove(initializerEvent.getUserPayload());
+    }
+    Assert.assertEquals(0, expectedPayloads.size());
+
+  }
+
+  @Test(timeout = 10000)
+  public void testInputInitializerEventNoDirectConnection() throws Exception {
+    useCustomInitializer = true;
+    customInitializer = new EventHandlingRootInputInitializer(null);
+    EventHandlingRootInputInitializer initializer =
+        (EventHandlingRootInputInitializer) customInitializer;
+    setupPreDagCreation();
+    dagPlan = createDAGPlanWithRunningInitializer4();
+    setupPostDagCreation();
+
+    VertexImplWithRunningInputInitializer v1 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+    VertexImplWithRunningInputInitializer v2 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+    VertexImplWithRunningInputInitializer v3 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+    initVertex(v1);
+    startVertex(v1);
+    dispatcher.await();
+
+    // Vertex1 start should trigger downstream vertices
+    Assert.assertEquals(VertexState.RUNNING, v1.getState());
+    Assert.assertEquals(VertexState.RUNNING, v2.getState());
+    Assert.assertEquals(VertexState.INITIALIZING, v3.getState());
+
+    // Genrate events from v1 to v3's InputInitializer
+    InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", null);
+    // Create taskId and taskAttemptId for the single task that exists in vertex1
+    TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+    TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+    TezEvent tezEvent = new TezEvent(event,
+        new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+    dispatcher.getEventHandler()
+        .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+    dispatcher.await();
+
+    // Events should not be cached in the vertex, since the initializer is running
+    Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+    // Events should be cached since the tasks have not succeeded.
+    // Verify that events are cached
+    RootInputInitializerManager.InitializerWrapper initializerWrapper =
+        v3.rootInputInitializerManager.getInitializerWrapper("input1");
+    Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+    Assert.assertEquals(1, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+    // Get all tasks of vertex1 to succeed.
+    for (TezTaskID taskId : v1.getTasks().keySet()) {
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+      v1.handle( new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+      v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+      dispatcher.await();
+      v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+    }
+    dispatcher.await();
+
+    // v3 would have processed an INIT event and moved into INITIALIZING state.
+    // Since source tasks were complete - the events should have been consumed.
+    // Initializer would have run, and processed events.
+    while (v3.getState()  != VertexState.RUNNING) {
+      Thread.sleep(10);
+    }
+
+    Assert.assertEquals(VertexState.RUNNING, v3.getState());
+
+    Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+    Assert.assertEquals(0, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+    Assert.assertTrue(initializer.eventReceived.get());
+    Assert.assertEquals(2, initializer.stateUpdates.size());
     Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.RUNNING,
-        initializer.stateUpdateEvents.get(0).getVertexState());
+        initializer.stateUpdates.get(0).getVertexState());
     Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.SUCCEEDED,
-        initializer.stateUpdateEvents.get(1).getVertexState());
+        initializer.stateUpdates.get(1).getVertexState());
+  }
+
+  @Test(timeout = 10000)
+  public void testInputInitializerEventsAtNew() throws Exception {
+    useCustomInitializer = true;
+    customInitializer = new EventHandlingRootInputInitializer(null);
+    EventHandlingRootInputInitializer initializer =
+        (EventHandlingRootInputInitializer) customInitializer;
+    setupPreDagCreation();
+    dagPlan = createDAGPlanWithRunningInitializer3();
+    setupPostDagCreation();
+
+    VertexImplWithRunningInputInitializer v1 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex1");
+    VertexImplWithRunningInputInitializer v2 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex2");
+    VertexImplWithRunningInputInitializer v3 =
+        (VertexImplWithRunningInputInitializer) vertices.get("vertex3");
+
+    initVertex(v1);
+    startVertex(v1);
+    dispatcher.await();
+
+    // Vertex2 has not been INITED, so the rest of the vertices should be in state NEW.
+    Assert.assertEquals(VertexState.RUNNING, v1.getState());
+    Assert.assertEquals(VertexState.NEW, v2.getState());
+    Assert.assertEquals(VertexState.NEW, v3.getState());
+
+    // Genrate events from v1 to v3's InputInitializer
+    InputInitializerEvent event = InputInitializerEvent.create("vertex3", "input1", null);
+    // Create taskId and taskAttemptId for the single task that exists in vertex1
+    TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+    TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
+    TezEvent tezEvent = new TezEvent(event,
+        new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex3", ta0_t0_v1));
+
+    dispatcher.getEventHandler()
+        .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
+    dispatcher.await();
+
+    // Events should be cached in the vertex, since the Initializer has not started
+    Assert.assertEquals(1, v3.pendingInitializerEvents.size());
+
+    // Get Vertex1 to succeed before Vertex2 is INITED. Contrived case ? This is likely a tiny race.
+    for (TezTaskID taskId : v1.getTasks().keySet()) {
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+      TaskImpl task = (TaskImpl)v1.getTask(taskId);
+      task.handle(new TaskEvent(taskId, TaskEventType.T_ATTEMPT_LAUNCHED));
+      task.handle(new TaskEventTAUpdate(taskAttemptId, TaskEventType.T_ATTEMPT_SUCCEEDED));
+      v1.handle(new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+      v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+      dispatcher.await();
+      v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+    }
+    dispatcher.await();
+
+    // Events should still be cached in the vertex
+    Assert.assertEquals(1, v3.pendingInitializerEvents.size());
+    Assert.assertEquals(VertexState.NEW, v3.getState());
+
+    // Move processing along. INIT the remaining root level vertex.
+    initVertex(v2);
+    startVertex(v2);
+    dispatcher.await();
+
+
+    // v3 would have processed an INIT event and moved into INITIALIZING state.
+    // Since source tasks were complete - the events should have been consumed.
+    // Initializer would have run, and processed events.
+    while (v3.getState()  != VertexState.RUNNING) {
+      Thread.sleep(10);
+    }
+
+    Assert.assertEquals(VertexState.RUNNING, v3.getState());
+    // Events should have been cleared from the vertex.
+    Assert.assertEquals(0, v3.pendingInitializerEvents.size());
+
+    // KK Add checks to validate thte RootInputManager doesn't remember the events either
+
+    Assert.assertTrue(initializer.eventReceived.get());
+    Assert.assertEquals(2, initializer.stateUpdates.size());
+    Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.RUNNING,
+        initializer.stateUpdates.get(0).getVertexState());
+    Assert.assertEquals(org.apache.tez.dag.api.event.VertexState.SUCCEEDED,
+        initializer.stateUpdates.get(1).getVertexState());
   }
 
   @SuppressWarnings("unchecked")
   @Test(timeout = 10000)
-  public void testRootInputInitializerEvent() throws Exception {
+  public void testInputInitializerEvents() throws Exception {
     useCustomInitializer = true;
     customInitializer = new EventHandlingRootInputInitializer(null);
     EventHandlingRootInputInitializer initializer =
@@ -2998,15 +3576,36 @@ public class TestVertexImpl {
     Assert.assertFalse(initializer.eventReceived.get());
     Assert.assertFalse(initializer.initComplete.get());
 
+
     // Signal the initializer by sending an event - via vertex1
     InputInitializerEvent event = InputInitializerEvent.create("vertex2", "input1", null);
+    // Create taskId and taskAttemptId for the single task that exists in vertex1
+    TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
+    TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
     TezEvent tezEvent = new TezEvent(event,
-        new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", null, null));
+        new EventMetaData(EventProducerConsumerType.OUTPUT, "vertex1", "vertex2", ta0_t0_v1));
 
     dispatcher.getEventHandler()
         .handle(new VertexEventRouteEvent(v1.getVertexId(), Collections.singletonList(tezEvent)));
     dispatcher.await();
 
+    // Events should not be cached in the vertex, since the initializer is running
+    Assert.assertEquals(0, v2.pendingInitializerEvents.size());
+
+    // Verify that events are cached
+    RootInputInitializerManager.InitializerWrapper initializerWrapper =
+        v2.rootInputInitializerManager.getInitializerWrapper("input1");
+    Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+    Assert.assertEquals(1, initializerWrapper.getPendingEvents().get(v1.getName()).size());
+
+    for (TezTaskID taskId : v1.getTasks().keySet()) {
+      TezTaskAttemptID taskAttemptId = TezTaskAttemptID.getInstance(taskId, 0);
+      v1.handle(new VertexEventTaskAttemptCompleted(taskAttemptId, TaskAttemptStateInternal.SUCCEEDED));
+      v1.handle(new VertexEventTaskCompleted(taskId, TaskState.SUCCEEDED));
+      dispatcher.await();
+      v1.stateChangeNotifier.taskSucceeded(v1.getName(), taskId, taskAttemptId.getId());
+    }
+
     // Both happening in separate threads
     while (!initializer.eventReceived.get()) {
       Thread.sleep(10);
@@ -3019,6 +3618,10 @@ public class TestVertexImpl {
     while (v2.getState()  != VertexState.RUNNING) {
       Thread.sleep(10);
     }
+
+    // Verify the events are no longer cached, but attempts are remembered
+    Assert.assertEquals(1, initializerWrapper.getFirstSuccessfulAttemptMap().size());
+    Assert.assertEquals(0, initializerWrapper.getPendingEvents().get(v1.getName()).size());
   }
 
   @Test(timeout = 5000)
@@ -3358,7 +3961,7 @@ public class TestVertexImpl {
     VertexImplWithControlledInitializerManager v2 = (VertexImplWithControlledInitializerManager) vertices.get("vertex2");
     Assert.assertEquals(VertexState.INITIALIZING, v2.getState());
     
-    // non-task events dont get buffered
+    // non-task events don't get buffered
     List<TezEvent> events = Lists.newLinkedList();
     TezTaskID t0_v1 = TezTaskID.getInstance(v1.getVertexId(), 0);
     TezTaskAttemptID ta0_t0_v1 = TezTaskAttemptID.getInstance(t0_v1, 0);
@@ -3954,8 +4557,10 @@ public class TestVertexImpl {
     private final ReentrantLock lock = new ReentrantLock();
     private final Condition eventCondition = lock.newCondition();
 
-    private final List<VertexStateUpdate> stateUpdateEvents = new LinkedList<VertexStateUpdate>();
+    private final List<VertexStateUpdate> stateUpdates = new LinkedList<VertexStateUpdate>();
+    private final List<InputInitializerEvent> initializerEvents = new LinkedList<InputInitializerEvent>();
     private volatile InputInitializerContext context;
+    private volatile int numExpectedEvents = 1;
 
     public EventHandlingRootInputInitializer(
         InputInitializerContext initializerContext) {
@@ -3968,7 +4573,9 @@ public class TestVertexImpl {
       initStarted.set(true);
       lock.lock();
       try {
-        eventCondition.await();
+        if (!eventReceived.get()) {
+          eventCondition.await();
+        }
       } finally {
         lock.unlock();
       }
@@ -3983,12 +4590,15 @@ public class TestVertexImpl {
     @Override
     public void handleInputInitializerEvent(List<InputInitializerEvent> events) throws
         Exception {
-      eventReceived.set(true);
-      lock.lock();
-      try {
-        eventCondition.signal();
-      } finally {
-        lock.unlock();
+      initializerEvents.addAll(events);
+      if (initializerEvents.size() == numExpectedEvents) {
+        eventReceived.set(true);
+        lock.lock();
+        try {
+          eventCondition.signal();
+        } finally {
+          lock.unlock();
+        }
       }
     }
 
@@ -3997,8 +4607,12 @@ public class TestVertexImpl {
       this.context = context;
     }
 
+    public void setNumExpectedEvents(int numEvents) {
+      this.numExpectedEvents = numEvents;
+    }
+
     public void onVertexStateUpdated(VertexStateUpdate stateUpdate) {
-      stateUpdateEvents.add(stateUpdate);
+      stateUpdates.add(stateUpdate);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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
index e2f189c..9042a93 100644
--- 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
@@ -62,7 +62,7 @@ import org.apache.tez.dag.app.dag.event.VertexEvent;
 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.TestVertexImpl.CountingOutputCommitter;
-import org.apache.tez.dag.history.events.VertexDataMovementEventsGeneratedEvent;
+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;
@@ -403,7 +403,7 @@ public class TestVertexRecovery {
   public void testRecovery_New_Desired_RUNNING() {
     VertexImpl vertex1 = (VertexImpl) dag.getVertex("vertex1");
     VertexState recoveredState =
-        vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
             vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
     assertEquals(VertexState.NEW, recoveredState);
     assertEquals(1, vertex1.recoveredEvents.size());
@@ -478,7 +478,7 @@ public class TestVertexRecovery {
     restoreFromInitializedEvent(vertex1);
 
     VertexState recoveredState =
-        vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
             vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
     assertEquals(VertexState.INITED, recoveredState);
 
@@ -522,7 +522,7 @@ public class TestVertexRecovery {
     assertEquals(startedTime, vertex1.startedTime);
 
     recoveredState =
-        vertex1.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+        vertex1.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
             vertex1.getVertexId(), Lists.newArrayList(createTezEvent())));
     assertEquals(VertexState.RUNNING, recoveredState);
     assertEquals(1, vertex1.recoveredEvents.size());
@@ -616,7 +616,7 @@ public class TestVertexRecovery {
 
     VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
     VertexState recoveredState =
-        vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
             vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
     assertEquals(VertexState.NEW, recoveredState);
     assertEquals(1, vertex3.recoveredEvents.size());
@@ -669,7 +669,7 @@ public class TestVertexRecovery {
 
     VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
     VertexState recoveredState =
-        vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
             vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
     assertEquals(VertexState.NEW, recoveredState);
     assertEquals(1, vertex3.recoveredEvents.size());
@@ -734,7 +734,7 @@ public class TestVertexRecovery {
 
     VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
     recoveredState =
-        vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
             vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
     assertEquals(VertexState.NEW, recoveredState);
     assertEquals(1, vertex3.recoveredEvents.size());
@@ -814,7 +814,7 @@ public class TestVertexRecovery {
 
     VertexImpl vertex3 = (VertexImpl) dag.getVertex("vertex3");
     recoveredState =
-        vertex3.restoreFromEvent(new VertexDataMovementEventsGeneratedEvent(
+        vertex3.restoreFromEvent(new VertexRecoverableEventsGeneratedEvent(
             vertex3.getVertexId(), Lists.newArrayList(createTezEvent())));
     assertEquals(VertexState.NEW, recoveredState);
     assertEquals(1, vertex3.recoveredEvents.size());

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 5bb7d35..bcbe6f1 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
@@ -527,9 +527,9 @@ public class TestHistoryEventsProtoConversion {
   }
 
   private void testVertexDataMovementEventsGeneratedEvent() throws Exception {
-    VertexDataMovementEventsGeneratedEvent event;
+    VertexRecoverableEventsGeneratedEvent event;
     try {
-      event = new VertexDataMovementEventsGeneratedEvent(
+      event = new VertexRecoverableEventsGeneratedEvent(
           TezVertexID.getInstance(
               TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 1), null);
       Assert.fail("Invalid creation should have errored out");
@@ -539,11 +539,11 @@ public class TestHistoryEventsProtoConversion {
     List<TezEvent> events =
         Arrays.asList(new TezEvent(DataMovementEvent.create(1, null),
             new EventMetaData(EventProducerConsumerType.SYSTEM, "foo", "bar", null)));
-    event = new VertexDataMovementEventsGeneratedEvent(
+    event = new VertexRecoverableEventsGeneratedEvent(
             TezVertexID.getInstance(
                 TezDAGID.getInstance(ApplicationId.newInstance(0, 1), 1), 1), events);
-    VertexDataMovementEventsGeneratedEvent deserializedEvent =
-        (VertexDataMovementEventsGeneratedEvent) testProtoConversion(event);
+    VertexRecoverableEventsGeneratedEvent deserializedEvent =
+        (VertexRecoverableEventsGeneratedEvent) testProtoConversion(event);
     Assert.assertEquals(event.getVertexID(), deserializedEvent.getVertexID());
     Assert.assertEquals(1,
         deserializedEvent.getTezEvents().size());

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 ad39531..f674fc0 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
@@ -45,7 +45,7 @@ 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.VertexDataMovementEventsGeneratedEvent;
+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;
@@ -152,7 +152,7 @@ public class TestHistoryEventJsonConversion {
           event = new ContainerStoppedEvent(containerId, random.nextInt(), -1, applicationAttemptId);
           break;
         case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-          event = new VertexDataMovementEventsGeneratedEvent();
+          event = new VertexRecoverableEventsGeneratedEvent();
           break;
         case DAG_COMMIT_STARTED:
           event = new DAGCommitStartedEvent();

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 5b19e80..b04b8d4 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
@@ -45,7 +45,7 @@ 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.VertexDataMovementEventsGeneratedEvent;
+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;
@@ -152,7 +152,7 @@ public class TestHistoryEventTimelineConversion {
           event = new ContainerStoppedEvent(containerId, random.nextInt(), -1, applicationAttemptId);
           break;
         case VERTEX_DATA_MOVEMENT_EVENTS_GENERATED:
-          event = new VertexDataMovementEventsGeneratedEvent();
+          event = new VertexRecoverableEventsGeneratedEvent();
           break;
         case DAG_COMMIT_STARTED:
           event = new DAGCommitStartedEvent();

http://git-wip-us.apache.org/repos/asf/tez/blob/b4580a7b/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 7676313..9595cb9 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
@@ -42,8 +42,8 @@ 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.VertexDataMovementEventsGeneratedEvent;
 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;
@@ -198,8 +198,8 @@ public class TestDAGRecovery {
             + ", eventType=" + historyEvent.getEventType()
             + ", event=" + historyEvent);
         if (historyEvent.getEventType() ==  HistoryEventType.VERTEX_DATA_MOVEMENT_EVENTS_GENERATED) {
-          VertexDataMovementEventsGeneratedEvent dmEvent =
-              (VertexDataMovementEventsGeneratedEvent)historyEvent;
+          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;


[20/25] git commit: TEZ-1571. Add create method for DataSinkDescriptor. Contributed by Jeff Zhang.

Posted by je...@apache.org.
TEZ-1571. Add create method for DataSinkDescriptor. Contributed by Jeff
Zhang.


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

Branch: refs/heads/TEZ-8
Commit: dbe6a38a5646841fa60e103566d3a54cc8f3065e
Parents: 5d6b8fd
Author: Siddharth Seth <ss...@apache.org>
Authored: Mon Sep 15 22:36:15 2014 -0700
Committer: Siddharth Seth <ss...@apache.org>
Committed: Mon Sep 15 22:36:15 2014 -0700

----------------------------------------------------------------------
 CHANGES.txt                                     |  1 +
 .../apache/tez/dag/api/DataSinkDescriptor.java  | 24 ++++++++++++++++++++
 .../org/apache/tez/dag/api/TestDAGVerify.java   | 10 ++++----
 .../tez/dag/app/dag/impl/TestDAGImpl.java       |  4 ++--
 .../tez/dag/history/utils/TestDAGUtils.java     |  4 ++--
 .../apache/tez/mapreduce/client/YARNRunner.java |  2 +-
 .../apache/tez/mapreduce/output/MROutput.java   |  2 +-
 .../mapreduce/examples/FilterLinesByWord.java   |  2 +-
 .../examples/FilterLinesByWordOneToOne.java     |  2 +-
 .../org/apache/tez/test/TestDAGRecovery2.java   |  2 +-
 10 files changed, 39 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/CHANGES.txt
----------------------------------------------------------------------
diff --git a/CHANGES.txt b/CHANGES.txt
index 1520e70..97cf79b 100644
--- a/CHANGES.txt
+++ b/CHANGES.txt
@@ -40,6 +40,7 @@ ALL CHANGES
   TEZ-1574. Support additional formats for the tez deployed archive
   TEZ-1563. TezClient.submitDAGSession alters DAG local resources regardless
   of DAG submission
+  TEZ-1571. Add create method for DataSinkDescriptor.
 
 Release 0.5.0: 2014-09-03
 

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
----------------------------------------------------------------------
diff --git a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
index 2240384..bc43c88 100644
--- a/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
+++ b/tez-api/src/main/java/org/apache/tez/dag/api/DataSinkDescriptor.java
@@ -51,6 +51,7 @@ public class DataSinkDescriptor {
    *          operation.
    * @param credentials Credentials needs to access the data sink
    */
+  @Deprecated
   public DataSinkDescriptor(OutputDescriptor outputDescriptor,
       @Nullable OutputCommitterDescriptor committerDescriptor,
       @Nullable Credentials credentials) {
@@ -59,6 +60,29 @@ public class DataSinkDescriptor {
     this.credentials = credentials;
   }
 
+  /**
+   * Create a {@link DataSinkDescriptor}
+   * @param outputDescriptor
+   *          An {@link OutputDescriptor} for the output
+   * @param committerDescriptor
+   *          Specify a committer to be used for the output. Can be null. After all
+   *          tasks in the vertex (or in the DAG) have completed, the committer
+   *          (if specified) is invoked to commit the outputs. Commit is a data
+   *          sink specific operation that usually determines the visibility of
+   *          the output to external observers. E.g. moving output files from
+   *          temporary dirs to the real output dir. When there are multiple
+   *          executions of a task, the commit process also helps decide which
+   *          execution will be included in the final output. Users should
+   *          consider whether their application or data sink need a commit
+   *          operation.
+   * @param credentials Credentials needs to access the data sink
+   */
+  public static DataSinkDescriptor create(OutputDescriptor outputDescriptor,
+      @Nullable OutputCommitterDescriptor committerDescriptor,
+      @Nullable Credentials credentials) {
+    return new DataSinkDescriptor(outputDescriptor, committerDescriptor, credentials);
+  }
+  
   public OutputDescriptor getOutputDescriptor() {
     return outputDescriptor;
   }

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
----------------------------------------------------------------------
diff --git a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
index bf9abf1..b0b6d29 100644
--- a/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
+++ b/tez-api/src/test/java/org/apache/tez/dag/api/TestDAGVerify.java
@@ -521,7 +521,7 @@ public class TestDAGVerify {
         ProcessorDescriptor.create("MapProcessor"),
         dummyTaskCount, dummyTaskResource);
     
-    v1.addDataSink("v2", new DataSinkDescriptor(null, null, null));
+    v1.addDataSink("v2", DataSinkDescriptor.create(null, null, null));
     
     Edge e1 = Edge.create(v1, v2,
         EdgeProperty.create(DataMovementType.SCATTER_GATHER,
@@ -545,7 +545,7 @@ public class TestDAGVerify {
         ProcessorDescriptor.create("MapProcessor"),
         dummyTaskCount, dummyTaskResource);
     
-    v1.addDataSink("v2", new DataSinkDescriptor(null, null, null));
+    v1.addDataSink("v2", DataSinkDescriptor.create(null, null, null));
     
     DAG dag = DAG.create("testDag");
     dag.addVertex(v1);
@@ -621,7 +621,7 @@ public class TestDAGVerify {
     DAG dag = DAG.create("testDag");
     VertexGroup uv12 = dag.createVertexGroup("uv12", v1, v2);
     OutputDescriptor outDesc = new OutputDescriptor();
-    uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, null, null));
+    uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null));
     
     GroupInputEdge e1 = GroupInputEdge.create(uv12, v3,
         EdgeProperty.create(DataMovementType.SCATTER_GATHER,
@@ -678,7 +678,7 @@ public class TestDAGVerify {
     String groupName1 = "uv12";
     VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2);
     OutputDescriptor outDesc = new OutputDescriptor();
-    uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, null, null));
+    uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null));
     
     String groupName2 = "uv23";
     VertexGroup uv23 = dag.createVertexGroup(groupName2, v2, v3);
@@ -760,7 +760,7 @@ public class TestDAGVerify {
     String groupName1 = "uv12";
     VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2);
     OutputDescriptor outDesc = new OutputDescriptor();
-    uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, null, null));
+    uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, null, null));
     
     String groupName2 = "uv23";
     VertexGroup uv23 = dag.createVertexGroup(groupName2, v2, v3);

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
index aba4fd9..03aedef 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/app/dag/impl/TestDAGImpl.java
@@ -364,8 +364,8 @@ public class TestDAGImpl {
         TotalCountingOutputCommitter.class.getName());
     org.apache.tez.dag.api.VertexGroup uv12 = dag.createVertexGroup(groupName1, v1, v2);
     OutputDescriptor outDesc = OutputDescriptor.create("output.class");
-    uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
-    v3.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
+    uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
+    v3.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
 
     GroupInputEdge e1 = GroupInputEdge.create(uv12, v3,
         EdgeProperty.create(DataMovementType.SCATTER_GATHER,

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
----------------------------------------------------------------------
diff --git a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java b/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
index 4a0120b..081ed10 100644
--- a/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
+++ b/tez-dag/src/test/java/org/apache/tez/dag/history/utils/TestDAGUtils.java
@@ -73,8 +73,8 @@ public class TestDAGUtils {
         .setHistoryText("uvOut HistoryText");
     OutputCommitterDescriptor ocd =
         OutputCommitterDescriptor.create(OutputCommitter.class.getName());
-    uv12.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
-    v3.addDataSink("uvOut", new DataSinkDescriptor(outDesc, ocd, null));
+    uv12.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
+    v3.addDataSink("uvOut", DataSinkDescriptor.create(outDesc, ocd, null));
 
     GroupInputEdge e1 = GroupInputEdge.create(uv12, v3,
         EdgeProperty.create(DataMovementType.SCATTER_GATHER,

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
index d6c8057..dfbf0cf 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/client/YARNRunner.java
@@ -428,7 +428,7 @@ public class YARNRunner implements ClientProtocol {
     if (stageNum == totalStages -1) {
       OutputDescriptor od = OutputDescriptor.create(MROutputLegacy.class.getName())
           .setUserPayload(vertexUserPayload);
-      vertex.addDataSink("MROutput", new DataSinkDescriptor(od,
+      vertex.addDataSink("MROutput", DataSinkDescriptor.create(od,
           OutputCommitterDescriptor.create(MROutputCommitter.class.getName()), null));
     }
 

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
----------------------------------------------------------------------
diff --git a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
index db9fbea..ab9b41d 100644
--- a/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
+++ b/tez-mapreduce/src/main/java/org/apache/tez/mapreduce/output/MROutput.java
@@ -179,7 +179,7 @@ public class MROutput extends AbstractLogicalOutput {
         }
       }
 
-      return new DataSinkDescriptor(
+      return DataSinkDescriptor.create(
           OutputDescriptor.create(outputClassName).setUserPayload(createUserPayload()),
           (doCommit ? OutputCommitterDescriptor.create(
               MROutputCommitter.class.getName()) : null), credentials);

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
index 9f533d9..8430c68 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWord.java
@@ -196,7 +196,7 @@ public class FilterLinesByWord extends Configured implements Tool {
         .setUserPayload(TezUtils.createUserPayloadFromConf(stage2Conf));
     OutputCommitterDescriptor ocd =
         OutputCommitterDescriptor.create(MROutputCommitter.class.getName());
-    stage2Vertex.addDataSink("MROutput", new DataSinkDescriptor(od, ocd, null));
+    stage2Vertex.addDataSink("MROutput", DataSinkDescriptor.create(od, ocd, null));
 
     UnorderedKVEdgeConfig edgeConf = UnorderedKVEdgeConfig
         .newBuilder(Text.class.getName(), TextLongPair.class.getName()).build();

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
index e038170..515cf70 100644
--- a/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
+++ b/tez-tests/src/main/java/org/apache/tez/mapreduce/examples/FilterLinesByWordOneToOne.java
@@ -185,7 +185,7 @@ public class FilterLinesByWordOneToOne extends Configured implements Tool {
     // Configure the Output for stage2
     stage2Vertex.addDataSink(
         "MROutput",
-        new DataSinkDescriptor(OutputDescriptor.create(MROutput.class.getName())
+        DataSinkDescriptor.create(OutputDescriptor.create(MROutput.class.getName())
             .setUserPayload(TezUtils.createUserPayloadFromConf(stage2Conf)),
             OutputCommitterDescriptor.create(MROutputCommitter.class.getName()), null));
 

http://git-wip-us.apache.org/repos/asf/tez/blob/dbe6a38a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
----------------------------------------------------------------------
diff --git a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
index 37bea80..b7957a3 100644
--- a/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
+++ b/tez-tests/src/test/java/org/apache/tez/test/TestDAGRecovery2.java
@@ -183,7 +183,7 @@ public class TestDAGRecovery2 {
             .toUserPayload())));
     OutputCommitterDescriptor ocd = OutputCommitterDescriptor.create(
         MultiAttemptDAG.FailingOutputCommitter.class.getName());
-    dag.getVertex("v3").addDataSink("FailingOutput", new DataSinkDescriptor(od, ocd, null));
+    dag.getVertex("v3").addDataSink("FailingOutput", DataSinkDescriptor.create(od, ocd, null));
     runDAGAndVerify(dag, State.FAILED);
   }